From 78a776ccade5b7ca7133651ef8aef75f5ecea501 Mon Sep 17 00:00:00 2001 From: arnikola Date: Sun, 27 Jan 2019 14:57:22 -0500 Subject: [PATCH] [query] Add graphite query path (#1308) Adds processing for carbon queries, handling the find and query endpoints. --- glide.lock | 11 +- glide.yaml | 3 + .../development/m3_stack/m3coordinator.yml | 2 - .../docker-integration-tests/carbon/test.sh | 4 +- .../m3coordinator/ingest/carbon/ingest.go | 35 +- .../ingest/carbon/ingest_test.go | 15 +- src/query/api/v1/handler/graphite/render.go | 197 ++ .../api/v1/handler/graphite/render_parser.go | 216 ++ .../api/v1/handler/graphite/render_test.go | 190 ++ src/query/api/v1/handler/graphite/search.go | 115 + .../api/v1/handler/graphite/search_parser.go | 124 + src/query/api/v1/httpd/handler.go | 10 + src/query/graphite/common/aggregation.go | 63 + src/query/graphite/common/aggregation_test.go | 51 + src/query/graphite/common/aliasing.go | 130 + src/query/graphite/common/basic_functions.go | 286 ++ .../graphite/common/basic_functions_test.go | 210 ++ src/query/graphite/common/bootstrap.go | 85 + src/query/graphite/common/context.go | 238 ++ src/query/graphite/common/context_test.go | 90 + src/query/graphite/common/engine.go | 70 + src/query/graphite/common/percentiles.go | 158 + src/query/graphite/common/percentiles_test.go | 114 + src/query/graphite/common/test_util.go | 161 + src/query/graphite/common/transform.go | 312 ++ src/query/graphite/common/transform_test.go | 380 +++ src/query/graphite/context/context.go | 164 + src/query/graphite/errors/errors.go | 99 + src/query/graphite/errors/errors_test.go | 39 + src/query/graphite/graphite/glob.go | 188 ++ src/query/graphite/graphite/glob_test.go | 109 + src/query/graphite/graphite/graphite_types.go | 263 ++ .../graphite/graphite/graphite_types_test.go | 170 + src/query/graphite/graphite/identify.go | 86 + src/query/graphite/graphite/identify_test.go | 72 + src/query/graphite/graphite/tags.go | 64 + src/query/graphite/graphite/tags_test.go | 35 + .../graphite/testdata/no-results.json | 367 +++ src/query/graphite/graphite/timespec.go | 127 + src/query/graphite/graphite/timespec_test.go | 106 + src/query/graphite/lexer/lexer.go | 483 +++ src/query/graphite/lexer/lexer_test.go | 162 + .../graphite/native/aggregation_functions.go | 416 +++ .../native/aggregation_functions_test.go | 455 +++ src/query/graphite/native/alias_functions.go | 49 + .../graphite/native/alias_functions_test.go | 219 ++ .../graphite/native/builtin_functions.go | 1951 +++++++++++ .../graphite/native/builtin_functions_test.go | 2868 +++++++++++++++++ src/query/graphite/native/compiler.go | 290 ++ src/query/graphite/native/compiler_test.go | 450 +++ src/query/graphite/native/engine.go | 65 + src/query/graphite/native/engine_test.go | 175 + src/query/graphite/native/expression.go | 181 ++ src/query/graphite/native/functions.go | 576 ++++ src/query/graphite/native/functions_test.go | 125 + src/query/graphite/native/summarize.go | 193 ++ src/query/graphite/native/summarize_test.go | 134 + src/query/graphite/stats/statistics.go | 200 ++ src/query/graphite/stats/statistics_test.go | 67 + src/query/graphite/storage/converter.go | 101 + src/query/graphite/storage/converter_test.go | 75 + src/query/graphite/storage/m3_wrapper.go | 156 + src/query/graphite/storage/m3_wrapper_test.go | 157 + src/query/graphite/storage/storage.go | 85 + src/query/graphite/testing/test_utils.go | 146 + .../graphite/ts/consolidation_approach.go | 64 + src/query/graphite/ts/datapoint.go | 96 + src/query/graphite/ts/lttb.go | 179 + src/query/graphite/ts/lttb_test.go | 695 ++++ src/query/graphite/ts/series.go | 644 ++++ src/query/graphite/ts/series_reducer.go | 64 + src/query/graphite/ts/series_test.go | 480 +++ src/query/graphite/ts/sortable_series.go | 81 + src/query/graphite/ts/sortable_series_test.go | 133 + src/query/graphite/ts/values.go | 249 ++ src/query/graphite/ts/values_test.go | 41 + src/query/storage/m3/multi_fetch_result.go | 24 + src/query/storage/m3/storage.go | 71 +- src/query/storage/m3/types.go | 13 +- .../test/m3/{storage.go => test_storage.go} | 0 src/query/ts/series.go | 33 +- 81 files changed, 17730 insertions(+), 75 deletions(-) create mode 100644 src/query/api/v1/handler/graphite/render.go create mode 100644 src/query/api/v1/handler/graphite/render_parser.go create mode 100644 src/query/api/v1/handler/graphite/render_test.go create mode 100644 src/query/api/v1/handler/graphite/search.go create mode 100644 src/query/api/v1/handler/graphite/search_parser.go create mode 100644 src/query/graphite/common/aggregation.go create mode 100644 src/query/graphite/common/aggregation_test.go create mode 100644 src/query/graphite/common/aliasing.go create mode 100644 src/query/graphite/common/basic_functions.go create mode 100644 src/query/graphite/common/basic_functions_test.go create mode 100644 src/query/graphite/common/bootstrap.go create mode 100644 src/query/graphite/common/context.go create mode 100644 src/query/graphite/common/context_test.go create mode 100644 src/query/graphite/common/engine.go create mode 100644 src/query/graphite/common/percentiles.go create mode 100644 src/query/graphite/common/percentiles_test.go create mode 100644 src/query/graphite/common/test_util.go create mode 100644 src/query/graphite/common/transform.go create mode 100644 src/query/graphite/common/transform_test.go create mode 100644 src/query/graphite/context/context.go create mode 100644 src/query/graphite/errors/errors.go create mode 100644 src/query/graphite/errors/errors_test.go create mode 100644 src/query/graphite/graphite/glob.go create mode 100644 src/query/graphite/graphite/glob_test.go create mode 100644 src/query/graphite/graphite/graphite_types.go create mode 100644 src/query/graphite/graphite/graphite_types_test.go create mode 100644 src/query/graphite/graphite/identify.go create mode 100644 src/query/graphite/graphite/identify_test.go create mode 100644 src/query/graphite/graphite/tags.go create mode 100644 src/query/graphite/graphite/tags_test.go create mode 100644 src/query/graphite/graphite/testdata/no-results.json create mode 100644 src/query/graphite/graphite/timespec.go create mode 100644 src/query/graphite/graphite/timespec_test.go create mode 100644 src/query/graphite/lexer/lexer.go create mode 100644 src/query/graphite/lexer/lexer_test.go create mode 100644 src/query/graphite/native/aggregation_functions.go create mode 100644 src/query/graphite/native/aggregation_functions_test.go create mode 100644 src/query/graphite/native/alias_functions.go create mode 100644 src/query/graphite/native/alias_functions_test.go create mode 100644 src/query/graphite/native/builtin_functions.go create mode 100644 src/query/graphite/native/builtin_functions_test.go create mode 100644 src/query/graphite/native/compiler.go create mode 100644 src/query/graphite/native/compiler_test.go create mode 100644 src/query/graphite/native/engine.go create mode 100644 src/query/graphite/native/engine_test.go create mode 100644 src/query/graphite/native/expression.go create mode 100644 src/query/graphite/native/functions.go create mode 100644 src/query/graphite/native/functions_test.go create mode 100644 src/query/graphite/native/summarize.go create mode 100644 src/query/graphite/native/summarize_test.go create mode 100644 src/query/graphite/stats/statistics.go create mode 100644 src/query/graphite/stats/statistics_test.go create mode 100644 src/query/graphite/storage/converter.go create mode 100644 src/query/graphite/storage/converter_test.go create mode 100644 src/query/graphite/storage/m3_wrapper.go create mode 100644 src/query/graphite/storage/m3_wrapper_test.go create mode 100644 src/query/graphite/storage/storage.go create mode 100644 src/query/graphite/testing/test_utils.go create mode 100644 src/query/graphite/ts/consolidation_approach.go create mode 100644 src/query/graphite/ts/datapoint.go create mode 100644 src/query/graphite/ts/lttb.go create mode 100644 src/query/graphite/ts/lttb_test.go create mode 100644 src/query/graphite/ts/series.go create mode 100644 src/query/graphite/ts/series_reducer.go create mode 100644 src/query/graphite/ts/series_test.go create mode 100644 src/query/graphite/ts/sortable_series.go create mode 100644 src/query/graphite/ts/sortable_series_test.go create mode 100644 src/query/graphite/ts/values.go create mode 100644 src/query/graphite/ts/values_test.go rename src/query/test/m3/{storage.go => test_storage.go} (100%) diff --git a/glide.lock b/glide.lock index ab17e6d2d2..b767231f8d 100644 --- a/glide.lock +++ b/glide.lock @@ -1,5 +1,5 @@ -hash: 3b9f18246edf191b3b76191c4f5b17dfaab32830c8c6fbac9c8f1699bad13b3f -updated: 2019-01-20T14:42:37.326427-05:00 +hash: 1436a6b7e82df04aa1504b672fa149052f24bf2ec588d6fac40231d512745074 +updated: 2019-01-25T17:12:33.864742-05:00 imports: - name: github.com/apache/thrift version: c2fb1c4e8c931d22617bebb0bf388cb4d5e6fcff @@ -200,6 +200,8 @@ imports: - json/parser - json/scanner - json/token +- name: github.com/hydrogen18/stalecucumber + version: 9b38526d4bdf8e197c31344777fc28f7f48d250d - name: github.com/inconshreveable/mousetrap version: 76626ae9c91c4f2a10f34cad8ce83ea42c93bb75 - name: github.com/jonboulle/clockwork @@ -246,6 +248,7 @@ imports: - tcp - test - time + - unsafe - watch - name: github.com/m3db/prometheus_client_golang version: 8ae269d24972b8695572fa6b2e3718b5ea82d6b4 @@ -295,7 +298,7 @@ imports: - name: github.com/pborman/uuid version: adf5a7427709b9deb95d29d3fa8a2bf9cfd388f1 - name: github.com/pelletier/go-buffruneio - version: e2f66f8164ca709d4c21e815860afd2024e9b894 + version: de1592c34d9c6055a32fc9ebe2b3ee50ca468ebe - name: github.com/pelletier/go-toml version: 3b00596b2e9ee541bbd72dc50cc0c60e2b46c69c - name: github.com/pilosa/pilosa @@ -540,6 +543,6 @@ testImports: - name: github.com/philhofer/fwd version: bb6d471dc95d4fe11e432687f8b70ff496cf3136 - name: github.com/tinylib/msgp - version: 53e4ad1e134ee9b42a7add28ca77177ab17983b3 + version: f65876d3ea05943d6613bca6b8ed391843044bd4 subpackages: - msgp diff --git a/glide.yaml b/glide.yaml index 2ce8318fe2..b61cc9197a 100644 --- a/glide.yaml +++ b/glide.yaml @@ -171,6 +171,9 @@ import: subpackages: - cmp + - package: github.com/hydrogen18/stalecucumber + version: 9b38526d4bdf8e197c31344777fc28f7f48d250d + # START_PROMETHEUS_DEPS - package: github.com/prometheus/prometheus version: 998dfcbac689ae832ea64ca134fcb096f61a7f62 diff --git a/scripts/development/m3_stack/m3coordinator.yml b/scripts/development/m3_stack/m3coordinator.yml index 67f2db2986..ec9d90f462 100644 --- a/scripts/development/m3_stack/m3coordinator.yml +++ b/scripts/development/m3_stack/m3coordinator.yml @@ -68,5 +68,3 @@ ingest: handler: protobufEnabled: true -carbon: - enabled: true diff --git a/scripts/docker-integration-tests/carbon/test.sh b/scripts/docker-integration-tests/carbon/test.sh index 0fdf580c3a..769ffada25 100755 --- a/scripts/docker-integration-tests/carbon/test.sh +++ b/scripts/docker-integration-tests/carbon/test.sh @@ -25,8 +25,8 @@ echo "foo.bar.baz 1 `date +%s`" | nc 0.0.0.0 7204 echo "Attempting to read carbon metric back" function read_carbon { end=$(date +%s) - start=$(($end-300)) - RESPONSE=$(curl -sSfg "http://localhost:7201/api/v1/query_range?start=$start&end=$end&step=10&query={__graphite0__='foo',__graphite1__='bar',__graphite2__='baz'}") + start=$(($end-3000)) + RESPONSE=$(curl -sSfg "http://localhost:7201/api/v1/query_range?start=$start&end=$end&step=10&query={__g0__='foo',__g1__='bar',__g2__='baz'}") echo "$RESPONSE" | jq '.data.result[0].values[][1]=="1"' | grep -q "true" return $? } diff --git a/src/cmd/services/m3coordinator/ingest/carbon/ingest.go b/src/cmd/services/m3coordinator/ingest/carbon/ingest.go index 6db162ccbb..e94cfcdc39 100644 --- a/src/cmd/services/m3coordinator/ingest/carbon/ingest.go +++ b/src/cmd/services/m3coordinator/ingest/carbon/ingest.go @@ -31,6 +31,7 @@ import ( "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest" "github.com/m3db/m3/src/metrics/carbon" + "github.com/m3db/m3/src/query/graphite/graphite" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/ts" "github.com/m3db/m3x/instrument" @@ -47,11 +48,6 @@ var ( carbonSeparatorByte = byte('.') carbonSeparatorBytes = []byte{carbonSeparatorByte} - // Number of pre-formatted key names to generate in the init() function. - numPreFormattedKeyNames = 100 - // Should never be modified after init(). - preFormattedKeyNames = [][]byte{} - errCannotGenerateTagsFromEmptyName = errors.New("cannot generate tags from empty name") errIOptsMustBeSet = errors.New("carbon ingester options: instrument options must be st") errWorkerPoolMustBeSet = errors.New("carbon ingester options: worker pool must be set") @@ -197,9 +193,9 @@ type carbonIngesterMetrics struct { // key-value pair tags such that an input like: // foo.bar.baz // becomes -// __graphite0__:foo -// __graphite1__:bar -// __graphite2__:baz +// __g0__:foo +// __g1__:bar +// __g2__:baz func GenerateTagsFromName(name []byte) (models.Tags, error) { if len(name) == 0 { return models.Tags{}, errCannotGenerateTagsFromEmptyName @@ -219,7 +215,7 @@ func GenerateTagsFromName(name []byte) (models.Tags, error) { } tags = append(tags, models.Tag{ - Name: getOrGenerateKeyName(tagNum), + Name: graphite.TagName(tagNum), Value: name[startIdx:i], }) startIdx = i + 1 @@ -238,29 +234,10 @@ func GenerateTagsFromName(name []byte) (models.Tags, error) { // then the foor loop would have appended foo, bar, and baz already. if name[len(name)-1] != carbonSeparatorByte { tags = append(tags, models.Tag{ - Name: getOrGenerateKeyName(tagNum), + Name: graphite.TagName(tagNum), Value: name[startIdx:], }) } return models.Tags{Tags: tags}, nil } - -func getOrGenerateKeyName(idx int) []byte { - if idx < len(preFormattedKeyNames) { - return preFormattedKeyNames[idx] - } - - return []byte(fmt.Sprintf("__graphite%d__", idx)) -} - -func generateKeyName(idx int) []byte { - return []byte(fmt.Sprintf("__graphite%d__", idx)) -} - -func init() { - for i := 0; i < numPreFormattedKeyNames; i++ { - keyName := generateKeyName(i) - preFormattedKeyNames = append(preFormattedKeyNames, keyName) - } -} diff --git a/src/cmd/services/m3coordinator/ingest/carbon/ingest_test.go b/src/cmd/services/m3coordinator/ingest/carbon/ingest_test.go index 4d36a617c4..9cd2ace5db 100644 --- a/src/cmd/services/m3coordinator/ingest/carbon/ingest_test.go +++ b/src/cmd/services/m3coordinator/ingest/carbon/ingest_test.go @@ -34,6 +34,7 @@ import ( "time" "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest" + "github.com/m3db/m3/src/query/graphite/graphite" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/ts" "github.com/m3db/m3x/instrument" @@ -110,23 +111,23 @@ func TestGenerateTagsFromName(t *testing.T) { { name: "foo", expectedTags: []models.Tag{ - {Name: []byte("__graphite0__"), Value: []byte("foo")}, + {Name: graphite.TagName(0), Value: []byte("foo")}, }, }, { name: "foo.bar.baz", expectedTags: []models.Tag{ - {Name: []byte("__graphite0__"), Value: []byte("foo")}, - {Name: []byte("__graphite1__"), Value: []byte("bar")}, - {Name: []byte("__graphite2__"), Value: []byte("baz")}, + {Name: graphite.TagName(0), Value: []byte("foo")}, + {Name: graphite.TagName(1), Value: []byte("bar")}, + {Name: graphite.TagName(2), Value: []byte("baz")}, }, }, { name: "foo.bar.baz.", expectedTags: []models.Tag{ - {Name: []byte("__graphite0__"), Value: []byte("foo")}, - {Name: []byte("__graphite1__"), Value: []byte("bar")}, - {Name: []byte("__graphite2__"), Value: []byte("baz")}, + {Name: graphite.TagName(0), Value: []byte("foo")}, + {Name: graphite.TagName(1), Value: []byte("bar")}, + {Name: graphite.TagName(2), Value: []byte("baz")}, }, }, { diff --git a/src/query/api/v1/handler/graphite/render.go b/src/query/api/v1/handler/graphite/render.go new file mode 100644 index 0000000000..bcc9331430 --- /dev/null +++ b/src/query/api/v1/handler/graphite/render.go @@ -0,0 +1,197 @@ +// Copyright (c) 2019 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 graphite + +import ( + "context" + "fmt" + "math" + "net/http" + "sort" + "sync" + + "github.com/m3db/m3/src/query/api/v1/handler" + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/native" + graphite "github.com/m3db/m3/src/query/graphite/storage" + "github.com/m3db/m3/src/query/graphite/ts" + "github.com/m3db/m3/src/query/storage" + "github.com/m3db/m3/src/x/net/http" +) + +const ( + // ReadURL is the url for the graphite query handler. + ReadURL = handler.RoutePrefixV1 + "/graphite/render" +) + +var ( + // ReadHTTPMethods is the HTTP methods used with this resource. + ReadHTTPMethods = []string{http.MethodGet, http.MethodPost} +) + +// A renderHandler implements the graphite /render endpoint, including full +// support for executing functions. It only works against data in M3. +type renderHandler struct { + engine *native.Engine +} + +type respError struct { + err error + code int +} + +// NewRenderHandler returns a new render handler around the given storage. +func NewRenderHandler( + storage storage.Storage, +) http.Handler { + wrappedStore := graphite.NewM3WrappedStorage(storage) + return &renderHandler{ + engine: native.NewEngine(wrappedStore), + } +} + +func sendError(errorCh chan error, err error) { + select { + case errorCh <- err: + default: + } +} + +// ServeHTTP processes the render requests. +func (h *renderHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + respErr := h.serveHTTP(w, r) + if respErr.err != nil { + xhttp.Error(w, respErr.err, respErr.code) + } +} + +func (h *renderHandler) serveHTTP( + w http.ResponseWriter, + r *http.Request, +) respError { + reqCtx := context.WithValue(r.Context(), handler.HeaderKey, r.Header) + p, err := ParseRenderRequest(r) + if err != nil { + return respError{err: err, code: http.StatusBadRequest} + } + + var ( + results = make([]ts.SeriesList, len(p.Targets)) + errorCh = make(chan error, 1) + mu sync.Mutex + ) + + ctx := common.NewContext(common.ContextOptions{ + Engine: h.engine, + Start: p.From, + End: p.Until, + Timeout: p.Timeout, + }) + + // Set the request context. + ctx.SetRequestContext(reqCtx) + defer ctx.Close() + + var wg sync.WaitGroup + wg.Add(len(p.Targets)) + for i, target := range p.Targets { + i, target := i, target + go func() { + // Log the query that causes us to panic. + defer func() { + if err := recover(); err != nil { + panic(fmt.Errorf("panic executing query '%s': %v", target, err)) + } + }() + + childCtx := ctx.NewChildContext(common.NewChildContextOptions()) + defer func() { + childCtx.Close() + wg.Done() + }() + + exp, err := h.engine.Compile(target) + if err != nil { + sendError(errorCh, errors.NewRenamedError(err, + fmt.Errorf("invalid 'target': %s => %s", target, err))) + return + } + + targetSeries, err := exp.Execute(childCtx) + if err != nil { + sendError(errorCh, errors.NewRenamedError(err, + fmt.Errorf("error: target %s returned %s", target, err))) + return + } + + for i, s := range targetSeries.Values { + if s.Len() <= int(p.MaxDataPoints) { + continue + } + + var ( + samplingMultiplier = math.Ceil(float64(s.Len()) / float64(p.MaxDataPoints)) + newMillisPerStep = int(samplingMultiplier * float64(s.MillisPerStep())) + ) + targetSeries.Values[i] = ts.LTTB(s, s.StartTime(), s.EndTime(), newMillisPerStep) + } + + mu.Lock() + results[i] = targetSeries + mu.Unlock() + }() + } + + wg.Wait() + close(errorCh) + err = <-errorCh + if err != nil { + return respError{err: err, code: http.StatusInternalServerError} + } + + // Count and sort the groups if not sorted already. + // NB(r): For certain things like stacking different targets in Grafana + // returning targets in order matters to give a deterministic order for + // the series to display when stacking. However we should only mutate + // the order if no expressions have explicitly applied their own sort. + numSeries := 0 + for _, r := range results { + numSeries += r.Len() + if !r.SortApplied { + sort.Stable(ts.SeriesByName(r.Values)) + } + } + + series := make([]*ts.Series, 0, numSeries) + for _, r := range results { + series = append(series, r.Values...) + } + + // We've always sorted the response by this point + response := ts.SeriesList{ + Values: series, + SortApplied: true, + } + + err = WriteRenderResponse(w, response) + return respError{err: err, code: http.StatusOK} +} diff --git a/src/query/api/v1/handler/graphite/render_parser.go b/src/query/api/v1/handler/graphite/render_parser.go new file mode 100644 index 0000000000..5fc5d6210c --- /dev/null +++ b/src/query/api/v1/handler/graphite/render_parser.go @@ -0,0 +1,216 @@ +// Copyright (c) 2019 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 graphite + +import ( + "fmt" + "io" + "math" + "net/http" + "strconv" + "time" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/graphite/ts" + "github.com/m3db/m3/src/query/util/json" +) + +const ( + realTimeQueryThreshold = time.Minute + queryRangeShiftThreshold = 55 * time.Minute + queryRangeShift = 15 * time.Second +) + +var ( + errNoTarget = errors.NewInvalidParamsError(errors.New("no 'target' specified")) + errFromNotBeforeUntil = errors.NewInvalidParamsError(errors.New("'from' must come before 'until'")) +) + +// WriteRenderResponse writes the response to a render request +func WriteRenderResponse( + w http.ResponseWriter, + series ts.SeriesList, +) error { + w.Header().Set("Content-Type", "application/json") + return renderResultsJSON(w, series.Values) +} + +const ( + tzOffsetForAbsoluteTime = time.Duration(0) + maxTimeout = time.Minute + defaultTimeout = time.Second * 5 +) + +// RenderRequest are the arguments to a render call. +type RenderRequest struct { + Targets []string + Format string + From time.Time + Until time.Time + MaxDataPoints int64 + Compare time.Duration + Timeout time.Duration +} + +// ParseRenderRequest parses the arguments to a render call from an incoming request. +func ParseRenderRequest(r *http.Request) (RenderRequest, error) { + var ( + p RenderRequest + err error + now = time.Now() + ) + + if err = r.ParseForm(); err != nil { + return p, err + } + + p.Targets = r.Form["target"] + + if len(p.Targets) == 0 { + return p, errNoTarget + } + + fromString, untilString := r.FormValue("from"), r.FormValue("until") + if len(fromString) == 0 { + fromString = "-30min" + } + + if len(untilString) == 0 { + untilString = "now" + } + + if p.From, err = graphite.ParseTime( + fromString, + now, + tzOffsetForAbsoluteTime, + ); err != nil { + return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'from': %s", fromString)) + } + + if p.Until, err = graphite.ParseTime( + untilString, + now, + tzOffsetForAbsoluteTime, + ); err != nil { + return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'until': %s", untilString)) + } + + if !p.From.Before(p.Until) { + return p, errFromNotBeforeUntil + } + + // If this is a real-time query, and the query range is large enough, we shift the query + // range slightly to take into account the clock skew between the client's local time and + // the server's local time in order to take advantage of possibly higher-resolution data. + // In the future we could potentially distinguish absolute time and relative time and only + // use the time range for policy resolution, although we need to be careful when passing + // the range for cross-DC queries. + isRealTimeQuery := now.Sub(p.Until) < realTimeQueryThreshold + isLargeRangeQuery := p.Until.Sub(p.From) > queryRangeShiftThreshold + if isRealTimeQuery && isLargeRangeQuery { + p.From = p.From.Add(queryRangeShift) + p.Until = p.Until.Add(queryRangeShift) + } + + offset := r.FormValue("offset") + if len(offset) > 0 { + dur, err := graphite.ParseDuration(offset) + if err != nil { + err = errors.NewInvalidParamsError(err) + return p, errors.NewRenamedError(err, fmt.Errorf("invalid 'offset': %s", err)) + } + + p.Until = p.Until.Add(dur) + p.From = p.From.Add(dur) + } + + maxDataPointsString := r.FormValue("maxDataPoints") + if len(maxDataPointsString) != 0 { + p.MaxDataPoints, err = strconv.ParseInt(maxDataPointsString, 10, 64) + + if err != nil || p.MaxDataPoints < 1 { + return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'maxDataPoints': %s", maxDataPointsString)) + } + } else { + p.MaxDataPoints = math.MaxInt64 + } + + compareString := r.FormValue("compare") + + if compareFrom, err := graphite.ParseTime( + compareString, + p.From, + tzOffsetForAbsoluteTime, + ); err != nil && len(compareString) != 0 { + return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'compare': %s", compareString)) + } else if p.From.Before(compareFrom) { + return p, errors.NewInvalidParamsError(fmt.Errorf("'compare' must be in the past")) + } else { + p.Compare = compareFrom.Sub(p.From) + } + + timeout := r.FormValue("timeout") + if timeout != "" { + duration, err := time.ParseDuration(timeout) + if err != nil { + return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'timeout': %v", err)) + } + if duration > maxTimeout { + return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'timeout': greater than %v", maxTimeout)) + } + p.Timeout = duration + } else { + p.Timeout = defaultTimeout + } + + return p, nil +} + +func renderResultsJSON(w io.Writer, series []*ts.Series) error { + jw := json.NewWriter(w) + jw.BeginArray() + for _, s := range series { + jw.BeginObject() + jw.BeginObjectField("target") + jw.WriteString(s.Name()) + jw.BeginObjectField("datapoints") + jw.BeginArray() + + if !s.AllNaN() { + for i := 0; i < s.Len(); i++ { + timestamp, val := s.StartTimeForStep(i), s.ValueAt(i) + jw.BeginArray() + jw.WriteFloat64(val) + jw.WriteInt(int(timestamp.Unix())) + jw.EndArray() + } + } + + jw.EndArray() + jw.BeginObjectField("step_size_ms") + jw.WriteInt(s.MillisPerStep()) + + jw.EndObject() + } + jw.EndArray() + return jw.Close() +} diff --git a/src/query/api/v1/handler/graphite/render_test.go b/src/query/api/v1/handler/graphite/render_test.go new file mode 100644 index 0000000000..a6d57ef3c0 --- /dev/null +++ b/src/query/api/v1/handler/graphite/render_test.go @@ -0,0 +1,190 @@ +// Copyright (c) 2019 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 graphite + +import ( + "fmt" + "io/ioutil" + "net/http" + "net/http/httptest" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/models" + "github.com/m3db/m3/src/query/storage" + "github.com/m3db/m3/src/query/storage/mock" + "github.com/m3db/m3/src/query/ts" + + "github.com/stretchr/testify/require" +) + +func TestParseNoQuery(t *testing.T) { + mockStorage := mock.NewMockStorage() + handler := NewRenderHandler(mockStorage) + + recorder := httptest.NewRecorder() + handler.ServeHTTP(recorder, newGraphiteReadHTTPRequest(t)) + + res := recorder.Result() + require.Equal(t, 400, res.StatusCode) +} + +func TestParseQueryNoResults(t *testing.T) { + mockStorage := mock.NewMockStorage() + mockStorage.SetFetchResult(&storage.FetchResult{}, nil) + handler := NewRenderHandler(mockStorage) + + req := newGraphiteReadHTTPRequest(t) + req.URL.RawQuery = "target=foo.bar&from=-2h&until=now" + recorder := httptest.NewRecorder() + handler.ServeHTTP(recorder, req) + + res := recorder.Result() + require.Equal(t, 200, res.StatusCode) + + buf, err := ioutil.ReadAll(res.Body) + require.NoError(t, err) + require.Equal(t, []byte("[]"), buf) +} + +func TestParseQueryResults(t *testing.T) { + mockStorage := mock.NewMockStorage() + start := time.Now().Add(-30 * time.Minute) + resolution := 10 * time.Second + vals := ts.NewFixedStepValues(resolution, 3, 3, start) + tags := models.NewTags(0, nil) + tags = tags.AddTag(models.Tag{Name: graphite.TagName(0), Value: []byte("foo")}) + tags = tags.AddTag(models.Tag{Name: graphite.TagName(1), Value: []byte("bar")}) + seriesList := ts.SeriesList{ + ts.NewSeries("irrelevant_name", vals, tags), + } + for _, series := range seriesList { + series.SetResolution(resolution) + } + + mockStorage.SetFetchResult(&storage.FetchResult{SeriesList: seriesList}, nil) + handler := NewRenderHandler(mockStorage) + + req := newGraphiteReadHTTPRequest(t) + req.URL.RawQuery = fmt.Sprintf("target=foo.bar&from=%d&until=%d", + start.Unix(), start.Unix()+30) + recorder := httptest.NewRecorder() + handler.ServeHTTP(recorder, req) + + res := recorder.Result() + require.Equal(t, 200, res.StatusCode) + + buf, err := ioutil.ReadAll(res.Body) + require.NoError(t, err) + expected := fmt.Sprintf( + `[{"target":"foo.bar","datapoints":[[3.000000,%d],`+ + `[3.000000,%d],[3.000000,%d]],"step_size_ms":%d}]`, + start.Unix(), start.Unix()+10, start.Unix()+20, resolution/time.Millisecond) + + require.Equal(t, expected, string(buf)) +} + +func TestParseQueryResultsMaxDatapoints(t *testing.T) { + mockStorage := mock.NewMockStorage() + + startStr := "03/07/14" + endStr := "03/07/15" + start, err := graphite.ParseTime(startStr, time.Now(), 0) + require.NoError(t, err) + end, err := graphite.ParseTime(endStr, time.Now(), 0) + require.NoError(t, err) + + resolution := 10 * time.Second + vals := ts.NewFixedStepValues(resolution, 4, 4, start) + seriesList := ts.SeriesList{ + ts.NewSeries("a", vals, models.NewTags(0, nil)), + } + for _, series := range seriesList { + series.SetResolution(resolution) + } + + mockStorage.SetFetchResult(&storage.FetchResult{SeriesList: seriesList}, nil) + handler := NewRenderHandler(mockStorage) + + req := newGraphiteReadHTTPRequest(t) + req.URL.RawQuery = "target=foo.bar&from=" + startStr + "&until=" + endStr + "&maxDataPoints=1" + recorder := httptest.NewRecorder() + handler.ServeHTTP(recorder, req) + + res := recorder.Result() + require.Equal(t, 200, res.StatusCode) + + buf, err := ioutil.ReadAll(res.Body) + require.NoError(t, err) + + expected := fmt.Sprintf( + `[{"target":"a","datapoints":[[4.000000,%d]],"step_size_ms":%d}]`, + start.Unix(), end.Sub(start)/time.Millisecond) + + require.Equal(t, expected, string(buf)) +} + +func TestParseQueryResultsMultiTarget(t *testing.T) { + mockStorage := mock.NewMockStorage() + minsAgo := 12 + start := time.Now().Add(-1 * time.Duration(minsAgo) * time.Minute) + + resolution := 10 * time.Second + vals := ts.NewFixedStepValues(resolution, 3, 3, start) + seriesList := ts.SeriesList{ + ts.NewSeries("a", vals, models.NewTags(0, nil)), + } + for _, series := range seriesList { + series.SetResolution(resolution) + } + + mockStorage.SetFetchResult(&storage.FetchResult{SeriesList: seriesList}, nil) + handler := NewRenderHandler(mockStorage) + + req := newGraphiteReadHTTPRequest(t) + req.URL.RawQuery = fmt.Sprintf("target=foo.bar&target=baz.qux&from=%d&until=%d", + start.Unix(), start.Unix()+30) + recorder := httptest.NewRecorder() + handler.ServeHTTP(recorder, req) + + res := recorder.Result() + require.Equal(t, 200, res.StatusCode) + + buf, err := ioutil.ReadAll(res.Body) + require.NoError(t, err) + + expected := fmt.Sprintf( + `[{"target":"a","datapoints":[[3.000000,%d],`+ + `[3.000000,%d],[3.000000,%d]],"step_size_ms":%d},`+ + `{"target":"a","datapoints":[[3.000000,%d],`+ + `[3.000000,%d],[3.000000,%d]],"step_size_ms":%d}]`, + start.Unix(), start.Unix()+10, start.Unix()+20, resolution/time.Millisecond, + start.Unix(), start.Unix()+10, start.Unix()+20, resolution/time.Millisecond) + + require.Equal(t, expected, string(buf)) +} + +func newGraphiteReadHTTPRequest(t *testing.T) *http.Request { + req, err := http.NewRequest(ReadHTTPMethods[0], ReadURL, nil) + require.NoError(t, err) + return req +} diff --git a/src/query/api/v1/handler/graphite/search.go b/src/query/api/v1/handler/graphite/search.go new file mode 100644 index 0000000000..375f86910d --- /dev/null +++ b/src/query/api/v1/handler/graphite/search.go @@ -0,0 +1,115 @@ +// Copyright (c) 2019 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 graphite + +import ( + "bytes" + "context" + "net/http" + + "github.com/m3db/m3/src/query/api/v1/handler" + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/storage" + "github.com/m3db/m3/src/query/util/logging" + "github.com/m3db/m3/src/x/net/http" + + "go.uber.org/zap" +) + +const ( + // SearchURL is the url for searching graphite metrics. + SearchURL = handler.RoutePrefixV1 + "/graphite/metrics/find" +) + +var ( + // SearchHTTPMethods is the HTTP methods used with this resource. + SearchHTTPMethods = []string{http.MethodGet, http.MethodPost} +) + +type grahiteSearchHandler struct { + storage storage.Storage +} + +// NewSearchHandler returns a new instance of handler. +func NewSearchHandler( + storage storage.Storage, +) http.Handler { + return &grahiteSearchHandler{ + storage: storage, + } +} + +func (h *grahiteSearchHandler) ServeHTTP( + w http.ResponseWriter, + r *http.Request, +) { + ctx := context.WithValue(r.Context(), handler.HeaderKey, r.Header) + logger := logging.WithContext(ctx) + w.Header().Set("Content-Type", "application/json") + query, rErr := parseSearchParamsToQuery(r) + if rErr != nil { + xhttp.Error(w, rErr.Inner(), rErr.Code()) + return + } + + opts := storage.NewFetchOptions() + result, err := h.storage.FetchTags(ctx, query, opts) + if err != nil { + logger.Error("unable to complete tags", zap.Error(err)) + xhttp.Error(w, err, http.StatusBadRequest) + return + } + + partCount := graphite.CountMetricParts(query.Raw) + partName := graphite.TagName(partCount - 1) + seenMap := make(map[string]bool, len(result.Metrics)) + for _, m := range result.Metrics { + tags := m.Tags.Tags + index := 0 + // TODO: make this more performant by computing the index for the tag name. + for i, tag := range tags { + if bytes.Equal(partName, tag.Name) { + index = i + break + } + } + + value := tags[index].Value + // If this value has already been encountered, check if + if hadExtra, seen := seenMap[string(value)]; seen && hadExtra { + continue + } + + hasExtraParts := len(tags) > partCount + seenMap[string(value)] = hasExtraParts + } + + prefix := graphite.DropLastMetricPart(query.Raw) + if len(prefix) > 0 { + prefix += "." + } + + // TODO: Support multiple result types + if err = searchResultsJSON(w, prefix, seenMap); err != nil { + logger.Error("unable to print search results", zap.Error(err)) + xhttp.Error(w, err, http.StatusBadRequest) + } +} diff --git a/src/query/api/v1/handler/graphite/search_parser.go b/src/query/api/v1/handler/graphite/search_parser.go new file mode 100644 index 0000000000..1f9909c4a7 --- /dev/null +++ b/src/query/api/v1/handler/graphite/search_parser.go @@ -0,0 +1,124 @@ +// Copyright (c) 2019 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 graphite + +import ( + "fmt" + "io" + "net/http" + "time" + + "github.com/m3db/m3/src/query/errors" + "github.com/m3db/m3/src/query/graphite/graphite" + graphiteStorage "github.com/m3db/m3/src/query/graphite/storage" + "github.com/m3db/m3/src/query/storage" + "github.com/m3db/m3/src/query/util/json" + "github.com/m3db/m3/src/x/net/http" +) + +func parseSearchParamsToQuery(r *http.Request) ( + *storage.FetchQuery, + *xhttp.ParseError, +) { + values := r.URL.Query() + now := time.Now() + fromString, untilString := r.FormValue("from"), r.FormValue("until") + if len(fromString) == 0 { + fromString = "0" + } + + if len(untilString) == 0 { + untilString = "now" + } + + from, err := graphite.ParseTime( + fromString, + now, + tzOffsetForAbsoluteTime, + ) + + if err != nil { + return nil, xhttp.NewParseError(fmt.Errorf("invalid 'from': %s", fromString), + http.StatusBadRequest) + } + + until, err := graphite.ParseTime( + untilString, + now, + tzOffsetForAbsoluteTime, + ) + + if err != nil { + return nil, xhttp.NewParseError(fmt.Errorf("invalid 'until': %s", untilString), + http.StatusBadRequest) + } + + query := values.Get("query") + if query == "" { + return nil, xhttp.NewParseError(errors.ErrNoQueryFound, http.StatusBadRequest) + } + + matchers := graphiteStorage.TranslateQueryToMatchers(query) + return &storage.FetchQuery{ + Raw: query, + TagMatchers: matchers, + Start: from, + End: until, + Interval: 0, + }, nil +} + +func searchResultsJSON( + w io.Writer, + prefix string, + tags map[string]bool, +) error { + jw := json.NewWriter(w) + jw.BeginArray() + + for value, hasChildren := range tags { + leaf := 1 + if hasChildren { + leaf = 0 + } + jw.BeginObject() + + jw.BeginObjectField("id") + jw.WriteString(fmt.Sprintf("%s%s", prefix, value)) + + jw.BeginObjectField("text") + jw.WriteString(value) + + jw.BeginObjectField("leaf") + jw.WriteInt(leaf) + + jw.BeginObjectField("expandable") + jw.WriteInt(1 - leaf) + + jw.BeginObjectField("allowChildren") + jw.WriteInt(1 - leaf) + + jw.EndObject() + } + + jw.EndArray() + return jw.Close() +} diff --git a/src/query/api/v1/httpd/handler.go b/src/query/api/v1/httpd/handler.go index e7d4b0df21..d8803a2e91 100644 --- a/src/query/api/v1/httpd/handler.go +++ b/src/query/api/v1/httpd/handler.go @@ -32,6 +32,7 @@ import ( "github.com/m3db/m3/src/cmd/services/m3query/config" "github.com/m3db/m3/src/query/api/v1/handler" "github.com/m3db/m3/src/query/api/v1/handler/database" + "github.com/m3db/m3/src/query/api/v1/handler/graphite" m3json "github.com/m3db/m3/src/query/api/v1/handler/json" "github.com/m3db/m3/src/query/api/v1/handler/namespace" "github.com/m3db/m3/src/query/api/v1/handler/openapi" @@ -187,6 +188,15 @@ func (h *Handler) RegisterRoutes() error { logged(validator.NewPromDebugHandler(nativePromReadHandler, h.scope)).ServeHTTP, ).Methods(validator.PromDebugHTTPMethod) + // Graphite endpoints + h.router.HandleFunc(graphite.ReadURL, + logged(graphite.NewRenderHandler(h.storage)).ServeHTTP, + ).Methods(graphite.ReadHTTPMethods...) + + h.router.HandleFunc(graphite.SearchURL, + logged(graphite.NewSearchHandler(h.storage)).ServeHTTP, + ).Methods(graphite.SearchHTTPMethods...) + if h.clusterClient != nil { placementOpts := placement.HandlerOptions{ ClusterClient: h.clusterClient, diff --git a/src/query/graphite/common/aggregation.go b/src/query/graphite/common/aggregation.go new file mode 100644 index 0000000000..03978bda93 --- /dev/null +++ b/src/query/graphite/common/aggregation.go @@ -0,0 +1,63 @@ +// Copyright (c) 2019 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 common + +import ( + "math" + + "github.com/m3db/m3/src/query/graphite/ts" +) + +// Range distills down a set of inputs into the range of the series. +func Range(ctx *Context, series ts.SeriesList, renamer SeriesListRenamer) (*ts.Series, error) { + numSeries := series.Len() + if numSeries == 0 { + return nil, ErrEmptySeriesList + } + normalized, start, end, millisPerStep, err := Normalize(ctx, series) + if err != nil { + return nil, err + } + numSteps := ts.NumSteps(start, end, millisPerStep) + vals := ts.NewValues(ctx, millisPerStep, numSteps) + nan := math.NaN() + + for i := 0; i < numSteps; i++ { + minVal, maxVal := nan, nan + for j := 0; j < numSeries; j++ { + v := normalized.Values[j].ValueAt(i) + if math.IsNaN(v) { + continue + } + if math.IsNaN(minVal) || minVal > v { + minVal = v + } + if math.IsNaN(maxVal) || maxVal < v { + maxVal = v + } + } + if !math.IsNaN(minVal) && !math.IsNaN(maxVal) { + vals.SetValueAt(i, maxVal-minVal) + } + } + name := renamer(normalized) + return ts.NewSeries(ctx, name, start, vals), nil +} diff --git a/src/query/graphite/common/aggregation_test.go b/src/query/graphite/common/aggregation_test.go new file mode 100644 index 0000000000..5f5a8f5292 --- /dev/null +++ b/src/query/graphite/common/aggregation_test.go @@ -0,0 +1,51 @@ +// Copyright (c) 2019 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 common + +import ( + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/require" +) + +func TestRangeOfSeries(t *testing.T) { + ctx, input := NewConsolidationTestSeries(consolidationStartTime, consolidationEndTime, 30*time.Second) + defer ctx.Close() + + in := ts.SeriesList{Values: input} + + expectedStart := ctx.StartTime.Add(-30 * time.Second) + expectedStep := 10000 + rangeSeries, err := Range(ctx, in, func(series ts.SeriesList) string { + return "woot" + }) + require.Nil(t, err) + + expected := TestSeries{ + Name: "woot", + Data: []float64{0, 0, 0, 12, 12, 12, 14, 14, 14, 0, 0, 0}, + } + + CompareOutputsAndExpected(t, expectedStep, expectedStart, []TestSeries{expected}, []*ts.Series{rangeSeries}) +} diff --git a/src/query/graphite/common/aliasing.go b/src/query/graphite/common/aliasing.go new file mode 100644 index 0000000000..777f2448cf --- /dev/null +++ b/src/query/graphite/common/aliasing.go @@ -0,0 +1,130 @@ +// Copyright (c) 2019 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 common + +import ( + "fmt" + "regexp" + "strconv" + "strings" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +var ( + backReferenceRe = regexp.MustCompile(`\\\d+`) +) + +// Alias takes one metric or a wildcard seriesList and a string in quotes. +// Prints the string instead of the metric name in the legend. +func Alias(_ *Context, series ts.SeriesList, a string) (ts.SeriesList, error) { + renamed := make([]*ts.Series, series.Len()) + for i := range series.Values { + renamed[i] = series.Values[i].RenamedTo(a) + } + series.Values = renamed + return series, nil +} + +// AliasByMetric takes a seriesList and applies an alias derived from the base +// metric name. +func AliasByMetric(ctx *Context, series ts.SeriesList) (ts.SeriesList, error) { + renamed := make([]*ts.Series, series.Len()) + for i, s := range series.Values { + firstPart := strings.Split(s.Name(), ",")[0] + terms := strings.Split(firstPart, ".") + renamed[i] = s.RenamedTo(terms[len(terms)-1]) + } + series.Values = renamed + return series, nil +} + +// AliasByNode renames a time series result according to a subset of the nodes +// in its hierarchy. +func AliasByNode(_ *Context, seriesList ts.SeriesList, nodes ...int) (ts.SeriesList, error) { + renamed := make([]*ts.Series, 0, seriesList.Len()) + for _, series := range seriesList.Values { + name := series.Name() + left := strings.LastIndex(name, "(") + 1 + name = name[left:] + right := strings.IndexAny(name, ",)") + if right == -1 { + right = len(name) + } + nameParts := strings.Split(name[0:right], ".") + newNameParts := make([]string, 0, len(nodes)) + for _, node := range nodes { + // NB(jayp): graphite supports negative indexing, so we need to also! + if node < 0 { + node += len(nameParts) + } + if node < 0 || node >= len(nameParts) { + continue + } + newNameParts = append(newNameParts, nameParts[node]) + } + newName := strings.Join(newNameParts, ".") + newSeries := series.RenamedTo(newName) + renamed = append(renamed, newSeries) + } + seriesList.Values = renamed + return seriesList, nil +} + +// AliasSub runs series names through a regex search/replace. +func AliasSub(_ *Context, input ts.SeriesList, search, replace string) (ts.SeriesList, error) { + regex, err := regexp.Compile(search) + if err != nil { + return ts.SeriesList{}, err + } + + output := make([]*ts.Series, input.Len()) + for idx, series := range input.Values { + name := series.Name() + if submatches := regex.FindStringSubmatch(name); submatches == nil { + // if the pattern doesn't match, we don't change the series name. + output[idx] = series + } else { + // go regexp package doesn't support back-references, so we need to work around it. + newName := regex.ReplaceAllString(name, replace) + newName = backReferenceRe.ReplaceAllStringFunc(newName, func(matched string) string { + index, retErr := strconv.Atoi(matched[1:]) + if retErr != nil { + err = retErr + return "" + } + if index >= len(submatches) { + err = errors.NewInvalidParamsError(fmt.Errorf("invalid group reference in %s", replace)) + return "" + } + return submatches[index] + }) + if err != nil { + return ts.SeriesList{}, err + } + output[idx] = series.RenamedTo(newName) + } + } + + input.Values = output + return input, nil +} diff --git a/src/query/graphite/common/basic_functions.go b/src/query/graphite/common/basic_functions.go new file mode 100644 index 0000000000..2b807ade8a --- /dev/null +++ b/src/query/graphite/common/basic_functions.go @@ -0,0 +1,286 @@ +// Copyright (c) 2019 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 common + +import ( + "fmt" + "math" + "regexp" + "strconv" + "strings" + "time" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +var ( + // ErrNegativeCount occurs when the request count is < 0. + ErrNegativeCount = errors.NewInvalidParamsError(errors.New("n must be positive")) + // ErrEmptySeriesList occurs when a function requires a series as input + ErrEmptySeriesList = errors.NewInvalidParamsError(errors.New("empty series list")) + // ErrInvalidIntervalFormat occurs when invalid interval string encountered + ErrInvalidIntervalFormat = errors.NewInvalidParamsError(errors.New("invalid format")) + + reInterval *regexp.Regexp + + intervals = map[string]time.Duration{ + "s": time.Second, + "sec": time.Second, + "seconds": time.Second, + "m": time.Minute, + "min": time.Minute, + "minute": time.Minute, + "minutes": time.Minute, + "h": time.Hour, + "hr": time.Hour, + "hour": time.Hour, + "hours": time.Hour, + "d": time.Hour * 24, + "day": time.Hour * 24, + "days": time.Hour * 24, + "w": time.Hour * 24 * 7, + "week": time.Hour * 24 * 7, + "weeks": time.Hour * 24 * 7, + "mon": time.Hour * 24 * 30, + "month": time.Hour * 24 * 30, + "months": time.Hour * 24 * 30, + "y": time.Hour * 24 * 365, + "year": time.Hour * 24 * 365, + "years": time.Hour * 24 * 365, + } +) + +const ( + // MillisPerSecond is for millis per second + MillisPerSecond = 1000 + // SecondsPerMinute is for seconds per minute + SecondsPerMinute = 60 + // MillisPerMinute is for milliseconds per minute + MillisPerMinute = MillisPerSecond * SecondsPerMinute +) + +// SeriesListRenamer is a signature for renaming multiple series +// into a single name +type SeriesListRenamer func(series ts.SeriesList) string + +// SeriesRenamer is a signature for renaming a single series +type SeriesRenamer func(series *ts.Series) string + +// Head returns the first n elements of a series list or the entire list +func Head(series ts.SeriesList, n int) (ts.SeriesList, error) { + if n < 0 { + return ts.SeriesList{}, ErrNegativeCount + } + r := series.Values[:int(math.Min(float64(n), float64(series.Len())))] + series.Values = r + return series, nil +} + +// Identity returns datapoints where the value equals the timestamp of the datapoint. +func Identity(ctx *Context, name string) (ts.SeriesList, error) { + millisPerStep := int(MillisPerMinute) + numSteps := int(ctx.EndTime.Sub(ctx.StartTime) / time.Minute) + vals := ts.NewValues(ctx, millisPerStep, numSteps) + curTimeInSeconds := ctx.StartTime.Unix() + for i := 0; i < vals.Len(); i++ { + vals.SetValueAt(i, float64(curTimeInSeconds)) + curTimeInSeconds += SecondsPerMinute + } + newSeries := ts.NewSeries(ctx, name, ctx.StartTime, vals) + newSeries.Specification = fmt.Sprintf("identity(%q)", name) + return ts.SeriesList{Values: []*ts.Series{newSeries}}, nil +} + +// Normalize normalizes all input series to the same start time, step size, and end time. +func Normalize(ctx *Context, input ts.SeriesList) (ts.SeriesList, time.Time, time.Time, int, error) { + numSeries := input.Len() + if numSeries == 0 { + return ts.SeriesList{}, ctx.StartTime, ctx.EndTime, -1, errors.NewInvalidParamsError(ErrEmptySeriesList) + } + if numSeries == 1 { + return input, input.Values[0].StartTime(), input.Values[0].EndTime(), input.Values[0].MillisPerStep(), nil + } + + lcmMillisPerStep := input.Values[0].MillisPerStep() + minBegin, maxEnd := input.Values[0].StartTime(), input.Values[0].EndTime() + + for _, in := range input.Values[1:] { + lcmMillisPerStep = int(ts.Lcm(int64(lcmMillisPerStep), int64(in.MillisPerStep()))) + + if minBegin.After(in.StartTime()) { + minBegin = in.StartTime() + } + + if maxEnd.Before(in.EndTime()) { + maxEnd = in.EndTime() + } + } + + // Fix the right interval border to be divisible by interval step. + maxEnd = maxEnd.Add(-time.Duration( + int64(maxEnd.Sub(minBegin)/time.Millisecond)%int64(lcmMillisPerStep)) * time.Millisecond) + + numSteps := ts.NumSteps(minBegin, maxEnd, lcmMillisPerStep) + + results := make([]*ts.Series, input.Len()) + + for i, in := range input.Values { + if in.StartTime() == minBegin && in.MillisPerStep() == lcmMillisPerStep && in.Len() == numSteps { + results[i] = in + continue + } + + c := ts.NewConsolidation(ctx, minBegin, maxEnd, lcmMillisPerStep, ts.Avg) + c.AddSeries(in, ts.Avg) + results[i] = c.BuildSeries(in.Name(), ts.Finalize) + } + + input.Values = results + return input, minBegin, maxEnd, lcmMillisPerStep, nil +} + +// Count draws a horizontal line representing the number of nodes found in the seriesList. +func Count(ctx *Context, seriesList ts.SeriesList, renamer SeriesListRenamer) (ts.SeriesList, error) { + if seriesList.Len() == 0 { + numSteps := ctx.EndTime.Sub(ctx.StartTime).Minutes() + vals := ts.NewZeroValues(ctx, MillisPerMinute, int(numSteps)) + r := ts.SeriesList{ + Values: []*ts.Series{ts.NewSeries(ctx, renamer(seriesList), ctx.StartTime, vals)}, + } + return r, nil + } + + normalized, start, end, millisPerStep, err := Normalize(ctx, seriesList) + if err != nil { + return ts.SeriesList{}, err + } + numSteps := int(end.Sub(start) / (time.Duration(millisPerStep) * time.Millisecond)) + vals := ts.NewConstantValues(ctx, float64(normalized.Len()), numSteps, millisPerStep) + return ts.SeriesList{ + Values: []*ts.Series{ts.NewSeries(ctx, renamer(normalized), start, vals)}, + }, nil +} + +// ParseInterval parses an interval string and returns the corresponding duration. +func ParseInterval(s string) (time.Duration, error) { + if m := reInterval.FindStringSubmatch(strings.TrimSpace(s)); len(m) != 0 { + amount, err := strconv.ParseInt(m[1], 10, 32) + + if err != nil { + return 0, errors.NewInvalidParamsError(err) + } + + interval := intervals[strings.ToLower(m[2])] + return interval * time.Duration(amount), nil + } + + return 0, ErrInvalidIntervalFormat +} + +// ConstantLine draws a horizontal line at a specified value +func ConstantLine(ctx *Context, value float64) (*ts.Series, error) { + millisPerStep := int(ctx.EndTime.Sub(ctx.StartTime) / time.Millisecond) + if millisPerStep <= 0 { + err := fmt.Errorf("invalid boundary params: startTime=%v, endTime=%v", ctx.StartTime, ctx.EndTime) + return nil, err + } + name := fmt.Sprintf(FloatingPointFormat, value) + newSeries := ts.NewSeries(ctx, name, ctx.StartTime, ts.NewConstantValues(ctx, value, 2, millisPerStep)) + return newSeries, nil +} + +// ConstantSeries returns a new constant series with a granularity +// of one data point per second +func ConstantSeries(ctx *Context, value float64) (*ts.Series, error) { + // NB(jeromefroe): We use a granularity of one second to ensure that when multiple series + // are normalized the constant series will always have the smallest granularity and will + // not cause another series to be normalized to a greater granularity. + numSteps := int(ctx.EndTime.Sub(ctx.StartTime) / time.Second) + if numSteps <= 0 { + err := fmt.Errorf("invalid boundary params: startTime=%v, endTime=%v", ctx.StartTime, ctx.EndTime) + return nil, err + } + name := fmt.Sprintf(FloatingPointFormat, value) + newSeries := ts.NewSeries(ctx, name, ctx.StartTime, ts.NewConstantValues(ctx, value, numSteps, MillisPerSecond)) + return newSeries, nil +} + +// RemoveEmpty removes all series that have NaN data +func RemoveEmpty(ctx *Context, input ts.SeriesList) (ts.SeriesList, error) { + output := make([]*ts.Series, 0, input.Len()) + for _, series := range input.Values { + if series.AllNaN() { + continue + } + seriesHasData := false + for i := 0; i < series.Len(); i++ { + v := series.ValueAt(i) + if !math.IsNaN(v) { + seriesHasData = true + break + } + } + if seriesHasData { + output = append(output, series) + } + } + input.Values = output + return input, nil +} + +// Changed will output a 1 if the value changed or 0 if not +func Changed(ctx *Context, seriesList ts.SeriesList, renamer SeriesRenamer) (ts.SeriesList, error) { + results := make([]*ts.Series, 0, seriesList.Len()) + nan := math.NaN() + for _, series := range seriesList.Values { + previous := nan + numSteps := series.Len() + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + for i := 0; i < numSteps; i++ { + v := series.ValueAt(i) + if math.IsNaN(previous) { + previous = v + vals.SetValueAt(i, 0) + } else if !math.IsNaN(v) && previous != v { + previous = v + vals.SetValueAt(i, 1) + } else { + vals.SetValueAt(i, 0) + } + } + newSeries := ts.NewSeries(ctx, renamer(series), series.StartTime(), vals) + results = append(results, newSeries) + } + seriesList.Values = results + return seriesList, nil +} + +func init() { + intervalNames := make([]string, 0, len(intervals)) + + for name := range intervals { + intervalNames = append(intervalNames, name) + } + + reInterval = regexp.MustCompile(fmt.Sprintf("(?i)^([+-]?[0-9]+)(%s)$", + strings.Join(intervalNames, "|"))) +} diff --git a/src/query/graphite/common/basic_functions_test.go b/src/query/graphite/common/basic_functions_test.go new file mode 100644 index 0000000000..3cb0d06f12 --- /dev/null +++ b/src/query/graphite/common/basic_functions_test.go @@ -0,0 +1,210 @@ +// Copyright (c) 2019 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 common + +import ( + "math" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var ( + consolidationStartTime = time.Now().Truncate(time.Minute).Add(10 * time.Second) + consolidationEndTime = consolidationStartTime.Add(1 * time.Minute) +) + +type limitFunc func(series ts.SeriesList, n int) (ts.SeriesList, error) + +func validateOutputs(t *testing.T, step int, start time.Time, expected []TestSeries, actual []*ts.Series) { + require.Equal(t, len(expected), len(actual)) + + for i := range expected { + a, e := actual[i], expected[i].Data + + require.Equal(t, len(e), a.Len()) + + for step := 0; step < a.Len(); step++ { + v := a.ValueAt(step) + assert.Equal(t, e[step], v, "invalid value for %d", step) + } + + assert.Equal(t, expected[i].Name, a.Name()) + assert.Equal(t, step, a.MillisPerStep()) + assert.Equal(t, start, a.StartTime()) + } +} + +func TestLimitFunctions(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + var ( + // Values are not required in this test. + testInput = []TestSeries{ + {"foo", []float64{}}, {"bar", []float64{}}, + {"baz", []float64{}}, {"qux", []float64{}}, + } + + tests = []struct { + f limitFunc + n int + inputs []TestSeries + output []TestSeries + err error + }{ + {Head, 2, testInput, testInput[:2], nil}, + {Head, 100, testInput, testInput, nil}, + {Head, -2, testInput, nil, ErrNegativeCount}, + } + + startTime = time.Now() + step = 100 + ) + + for _, test := range tests { + series := ts.SeriesList{Values: NewTestSeriesList(ctx, startTime, test.inputs, step)} + output, err := test.f(series, test.n) + assert.Equal(t, err, test.err) + + validateOutputs(t, step, startTime, test.output, output.Values) + } +} + +func TestNormalize(t *testing.T) { + ctx, input := NewConsolidationTestSeries(consolidationStartTime, consolidationEndTime, 30*time.Second) + defer ctx.Close() + + normalized, start, end, step, err := Normalize(ctx, ts.SeriesList{ + Values: input, + }) + expectedStart := ctx.StartTime.Add(-30 * time.Second) + expectedEnd := ctx.StartTime.Add(90 * time.Second) + expectedStep := 10000 + nan := math.NaN() + require.Nil(t, err) + require.Equal(t, expectedStart, start) + require.Equal(t, expectedEnd, end) + require.Equal(t, expectedStep, step) + expected := []TestSeries{ + TestSeries{Name: "a", Data: []float64{nan, nan, nan, 10, 10, 10, 10, 10, 10, nan, nan, nan}}, + TestSeries{Name: "b", Data: []float64{15, 15, 15, 15, 15, 15, nan, nan, nan, nan, nan, nan}}, + TestSeries{Name: "c", Data: []float64{nan, nan, nan, nan, nan, nan, 17, 17, 17, 17, 17, 17}}, + TestSeries{Name: "d", Data: []float64{nan, nan, nan, 3, 3, 3, 3, 3, 3, nan, nan, nan}}, + } + + CompareOutputsAndExpected(t, expectedStep, expectedStart, expected, normalized.Values) +} + +func TestParseInterval(t *testing.T) { + tests := map[string]time.Duration{ + "5s": time.Second * 5, + "20sec": time.Second * 20, + "60seconds": time.Second * 60, + "1min": time.Minute * 1, + "10min": time.Minute * 10, + "2minutes": time.Minute * 2, + "3minute": time.Minute * 3, + "36h": time.Hour * 36, + "9hours": time.Hour * 9, + "1hour": time.Hour * 1, + "12hr": time.Hour * 12, + "1d": time.Hour * 24, + "2days": time.Hour * 24 * 2, + "1mon": time.Hour * 24 * 30, + "4W": time.Hour * 24 * 7 * 4, + "40weeks": time.Hour * 24 * 7 * 40, + "6months": time.Hour * 24 * 30 * 6, + "2y": time.Hour * 24 * 365 * 2, + "10years": time.Hour * 24 * 365 * 10, + } + + for s, d := range tests { + actual, err := ParseInterval(s) + require.NoError(t, err, "got error parsing interval %s", s) + + assert.Equal(t, d, actual, "invalid parse result for %s", s) + } + + actual, err := ParseInterval("122222222222222222222222222222s") + assert.True(t, errors.IsInvalidParams(err)) + assert.Equal(t, time.Duration(0), actual) +} + +func TestCount(t *testing.T) { + ctx, valid := NewConsolidationTestSeries(consolidationStartTime, consolidationEndTime, 30*time.Second) + defer ctx.Close() + + longCtx := NewContext(ContextOptions{ + Start: consolidationStartTime, + End: consolidationStartTime.Add(10 * time.Minute), + }) + defer longCtx.Close() + + renamer := func(series ts.SeriesList) string { + return "woot" + } + + tests := []struct { + context *Context + input []*ts.Series + renamer SeriesListRenamer + expectedSeries TestSeries + expectedStart time.Time + expectedStep int + }{{ + ctx, + valid, + renamer, + TestSeries{ + Name: "woot", + Data: []float64{4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, + }, + valid[1].StartTime(), + valid[1].MillisPerStep(), + }, + { + longCtx, + nil, + renamer, + TestSeries{ + "woot", + []float64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + }, + ctx.StartTime, + MillisPerMinute, + }, + } + + for _, test := range tests { + series := ts.SeriesList{Values: test.input} + results, err := Count(test.context, series, test.renamer) + require.Nil(t, err) + + CompareOutputsAndExpected(t, test.expectedStep, test.expectedStart, + []TestSeries{test.expectedSeries}, results.Values) + } +} diff --git a/src/query/graphite/common/bootstrap.go b/src/query/graphite/common/bootstrap.go new file mode 100644 index 0000000000..9e60ce7ec2 --- /dev/null +++ b/src/query/graphite/common/bootstrap.go @@ -0,0 +1,85 @@ +// Copyright (c) 2019 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 common + +import ( + "fmt" + "time" + + "github.com/m3db/m3/src/query/graphite/ts" +) + +// bootstrapWithIDs mocks fetches for now as the seriesList names are not actually IDs that are fetchable +// NaN vals will be returned for the period of startTime to EndTime +func bootstrapWithIDs(ctx *Context, seriesList ts.SeriesList, startTime, endTime time.Time) (ts.SeriesList, error) { + bootstrapList := make([]*ts.Series, seriesList.Len()) + + dur := int(endTime.Sub(startTime)) + for i, series := range seriesList.Values { + numSteps := dur / (series.MillisPerStep() * 1000 * 1000) // convert to ns for step calculation + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + bootstrapList[i] = ts.NewSeries(ctx, series.Name(), startTime, vals) + } + + seriesList.Values = bootstrapList + return seriesList, nil +} + +// FetchWithBootstrap requests the same data but with a bootstrap period at the beginning. +func FetchWithBootstrap(ctx *Context, seriesList ts.SeriesList, duration time.Duration) (ts.SeriesList, error) { + // Fetch bootstrapped series list between startTime and endTime + startTime := ctx.StartTime.Add(-duration) + endTime := ctx.StartTime + bootstrapList, err := bootstrapWithIDs(ctx, seriesList, startTime, endTime) + if err != nil { + return ts.SeriesList{}, fmt.Errorf("unable to fetch bootstrap series, error=%s", err) + } + + // Assemble the bootstrapped list + newSeriesList := make([]*ts.Series, seriesList.Len()) + for i, bootstrap := range bootstrapList.Values { + original := seriesList.Values[i] + if bootstrap.MillisPerStep() < original.MillisPerStep() { + bootstrap, err = bootstrap.IntersectAndResize(bootstrap.StartTime(), bootstrap.EndTime(), original.MillisPerStep(), original.ConsolidationFunc()) + if err != nil { + return ts.SeriesList{}, err + } + } + ratio := bootstrap.MillisPerStep() / original.MillisPerStep() + numBootstrapValues := bootstrap.Len() * ratio + numCombinedValues := numBootstrapValues + original.Len() + values := ts.NewValues(ctx, original.MillisPerStep(), numCombinedValues) + for j := 0; j < bootstrap.Len(); j++ { + for k := j * ratio; k < (j+1)*ratio; k++ { + values.SetValueAt(k, bootstrap.ValueAt(j)) + } + } + for j := numBootstrapValues; j < numCombinedValues; j++ { + values.SetValueAt(j, original.ValueAt(j-numBootstrapValues)) + } + newSeries := ts.NewSeries(ctx, original.Name(), startTime, values) + newSeries.Specification = original.Specification + newSeriesList[i] = newSeries + } + + seriesList.Values = newSeriesList + return seriesList, nil +} diff --git a/src/query/graphite/common/context.go b/src/query/graphite/common/context.go new file mode 100644 index 0000000000..d58c863596 --- /dev/null +++ b/src/query/graphite/common/context.go @@ -0,0 +1,238 @@ +// Copyright (c) 2019 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 common + +import ( + ctx "context" + "sync" + "time" + + "github.com/m3db/m3/src/query/graphite/context" +) + +// contextBase are the real content of a Context, minus the lock so that we +// can safely copy a context without violating the rules of go vet. +// nolint +type contextBase struct { + // TimeRangeAdjusted is a boolean indicating whether the time range has an adjustment. + TimeRangeAdjusted bool + + // The start time to query against. + StartTime time.Time + + // The end time to query against. + EndTime time.Time + + // TimeRangeAdjustment is the time range adjustment made to the query. + TimeRangeAdjustment TimeRangeAdjustment + + // The underlying engine. + Engine QueryEngine + + // Trace records traces. + Trace Tracer + + // Timeout indicates whether to use a custom timeout when fetching data, + // specify zero to indicate default timeout or a positive value + Timeout time.Duration + + parent *Context + reqCtx ctx.Context + storageContext context.Context +} + +// Context is the parameters to a query evaluation. +type Context struct { + sync.RWMutex + contextBase +} + +// ContextOptions provides the options to create the context with +type ContextOptions struct { + Start time.Time + End time.Time + Engine QueryEngine + Timeout time.Duration +} + +// TimeRangeAdjustment is an applied time range adjustment. +type TimeRangeAdjustment struct { + OriginalStart time.Time + OriginalEnd time.Time + ShiftStart time.Duration + ShiftEnd time.Duration + ExpandStart time.Duration + ExpandEnd time.Duration +} + +// NewContext creates a new context. +func NewContext(options ContextOptions) *Context { + return &Context{ + contextBase: contextBase{ + StartTime: options.Start, + EndTime: options.End, + Engine: options.Engine, + storageContext: context.New(), + Timeout: options.Timeout, + }, + } +} + +// TracingEnabled checks whether tracing is enabled for this context. +func (c *Context) TracingEnabled() bool { return c.Trace != nil } + +// ChildContextOptions is a set of options to pass when creating a child context. +type ChildContextOptions struct { + adjustment struct { + adjusted bool + shiftStart time.Duration + shiftEnd time.Duration + expandStart time.Duration + expandEnd time.Duration + } +} + +// NewChildContextOptions returns an initialized ChildContextOptions struct. +func NewChildContextOptions() ChildContextOptions { + return ChildContextOptions{} +} + +// AdjustTimeRange will adjust the child context's time range. +func (o *ChildContextOptions) AdjustTimeRange( + shiftStart, shiftEnd, expandStart, expandEnd time.Duration, +) { + if shiftStart == 0 && shiftEnd == 0 && expandStart == 0 && expandEnd == 0 { + // Not an adjustment, don't mark "adjusted" true + return + } + o.adjustment.adjusted = true + o.adjustment.shiftStart = shiftStart + o.adjustment.shiftEnd = shiftEnd + o.adjustment.expandStart = expandStart + o.adjustment.expandEnd = expandEnd +} + +// NewChildContext creates a child context. Child contexts can have any of +// their parameters modified, but share the same underlying storage context. +func (c *Context) NewChildContext(opts ChildContextOptions) *Context { + // create a duplicate of the parent context with an independent lock + // (otherwise `go vet` complains due to the -copylock check) + c.RLock() + child := &Context{ + contextBase: c.contextBase, + } + child.parent = c + c.RUnlock() + + origStart, origEnd := child.StartTime, child.EndTime + if child.TimeRangeAdjusted { + origStart, origEnd = c.TimeRangeAdjustment.OriginalStart, c.TimeRangeAdjustment.OriginalEnd + } + + if opts.adjustment.adjusted { + child.TimeRangeAdjusted = true + child.TimeRangeAdjustment.OriginalStart = origStart + child.TimeRangeAdjustment.OriginalEnd = origEnd + child.TimeRangeAdjustment.ShiftStart += opts.adjustment.shiftStart + child.TimeRangeAdjustment.ShiftEnd += opts.adjustment.shiftEnd + if opts.adjustment.expandStart > child.TimeRangeAdjustment.ExpandStart { + child.TimeRangeAdjustment.ExpandStart = opts.adjustment.expandStart + } + if opts.adjustment.expandEnd > child.TimeRangeAdjustment.ExpandEnd { + child.TimeRangeAdjustment.ExpandEnd = opts.adjustment.expandEnd + } + + child.StartTime = origStart. + Add(child.TimeRangeAdjustment.ShiftStart). + Add(-child.TimeRangeAdjustment.ExpandStart) + child.EndTime = origEnd. + Add(child.TimeRangeAdjustment.ShiftEnd). + Add(child.TimeRangeAdjustment.ExpandEnd) + } + + child.reqCtx = c.reqCtx + return child +} + +// Close closes the context +func (c *Context) Close() error { + if c.parent != nil { + // Closing a child context is meaningless. + return nil + } + + return c.storageContext.Close() +} + +// SetRequestContext sets the given context as the request context for this +// execution context. This is used for calls to the m3 storage wrapper. +func (c *Context) SetRequestContext(reqCtx ctx.Context) { + c.Lock() + c.reqCtx = reqCtx + c.Unlock() +} + +// RequestContext will provide the wrapped request context. Used for calls +// to m3 storage wrapper. +func (c *Context) RequestContext() ctx.Context { + c.RLock() + r := c.reqCtx + c.RUnlock() + return r +} + +// RegisterCloser registers a new Closer with the context. +func (c *Context) RegisterCloser(closer context.Closer) { + c.storageContext.RegisterCloser(closer) +} + +// AddAsyncTasks adds tracked asynchronous task(s) +func (c *Context) AddAsyncTasks(count int) { + c.storageContext.AddAsyncTasks(count) +} + +// DoneAsyncTask marks a single tracked asynchronous task complete +func (c *Context) DoneAsyncTask() { + c.storageContext.DoneAsyncTask() +} + +// A Trace is tracing information about a function or fetch within a query. +type Trace struct { + // ActivityName is the name of the activity being traced. + ActivityName string + + // Duration is the amount of time it took to execute the activity. + Duration time.Duration + + // Inputs are the number of timeseries processed by the trace. + Inputs []TraceStats + + // Outputs is the number of timeseries returned by the trace. + Outputs TraceStats +} + +// TraceStats tracks the number of timeseries used by a trace. +type TraceStats struct { + NumSeries int // number of timeseries being acted on +} + +// A Tracer is used to record a Trace. +type Tracer func(t Trace) diff --git a/src/query/graphite/common/context_test.go b/src/query/graphite/common/context_test.go new file mode 100644 index 0000000000..ee55c8f646 --- /dev/null +++ b/src/query/graphite/common/context_test.go @@ -0,0 +1,90 @@ +// Copyright (c) 2019 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 common + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" +) + +type closerFunc func() error + +func (f closerFunc) Close() error { + return f() +} + +func TestChildContext(t *testing.T) { + var ( + ctx = NewTestContext() + childCtx = ctx.NewChildContext(NewChildContextOptions()) + called = false + ) + + ctx.RegisterCloser(closerFunc(func() error { + called = true + return nil + })) + + childCtx.Close() + assert.False(t, called, "child context has closed the context root") + + ctx.Close() + assert.True(t, called, "parent context hasn't closed the context root") +} + +type mockClient struct { + mock.Mock +} + +func (m *mockClient) foo() { + m.Called() +} + +func (m *mockClient) Close() error { + m.foo() + return nil +} + +func TestContextClose(t *testing.T) { + client := &mockClient{} + client.On("foo").Return() + engine := NewEngine(nil) + ctx := NewContext(ContextOptions{Start: time.Now(), End: time.Now(), Engine: engine}) + ctx.RegisterCloser(client) + ctx.Close() + client.AssertCalled(t, "foo") +} + +func TestChildContextClose(t *testing.T) { + client := &mockClient{} + client.On("foo").Return() + engine := NewEngine(nil) + ctx := NewContext(ContextOptions{Start: time.Now(), End: time.Now(), Engine: engine}) + childContext := ctx.NewChildContext(NewChildContextOptions()) + childContext.RegisterCloser(client) + childContext.Close() + client.AssertNotCalled(t, "foo") + ctx.Close() + client.AssertCalled(t, "foo") +} diff --git a/src/query/graphite/common/engine.go b/src/query/graphite/common/engine.go new file mode 100644 index 0000000000..8a645feb39 --- /dev/null +++ b/src/query/graphite/common/engine.go @@ -0,0 +1,70 @@ +// Copyright (c) 2019 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 common + +import ( + "time" + + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/storage" +) + +// QueryEngine is the generic engine interface. +type QueryEngine interface { + FetchByQuery( + ctx context.Context, + query string, + start, end time.Time, + timeout time.Duration, + ) (*storage.FetchResult, error) +} + +// The Engine for running queries +type Engine struct { + storage storage.Storage +} + +// NewEngine creates a new query engine +func NewEngine(storage storage.Storage) *Engine { + return &Engine{ + storage: storage, + } +} + +// FetchByQuery retrieves one or more time series based on a query +func (e *Engine) FetchByQuery( + ctx context.Context, + query string, + start, end time.Time, + timeout time.Duration, +) (*storage.FetchResult, error) { + return e.storage.FetchByQuery( + ctx, + query, + storage.FetchOptions{ + StartTime: start, + EndTime: end, + DataOptions: storage.DataOptions{ + Timeout: timeout, + }, + }, + ) +} diff --git a/src/query/graphite/common/percentiles.go b/src/query/graphite/common/percentiles.go new file mode 100644 index 0000000000..712be1b47c --- /dev/null +++ b/src/query/graphite/common/percentiles.go @@ -0,0 +1,158 @@ +// Copyright (c) 2019 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 common + +import ( + "fmt" + "math" + "sort" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +const ( + // FloatingPointFormat is the floating point format for naming + FloatingPointFormat = "%.3f" +) + +// ErrInvalidPercentile is used when the percentile specified is incorrect +func ErrInvalidPercentile(percentile float64) error { + return errors.NewInvalidParamsError(fmt.Errorf("invalid percentile, percentile="+FloatingPointFormat, percentile)) +} + +// PercentileNamer formats a string with a percentile +type PercentileNamer func(name string, percentile float64) string + +// ThresholdComparator compares two floats for other comparison +// functions such as Percentile checks. +type ThresholdComparator func(v, threshold float64) bool + +// GreaterThan is a ThresholdComparator function for when +// a value is greater than a threshold +func GreaterThan(v, threshold float64) bool { + return v > threshold +} + +// LessThan is a ThresholdComparator function for when +// a value is less than a threshold +func LessThan(v, threshold float64) bool { + return v < threshold +} + +// SafeSort sorts the input slice and returns the number of NaNs in the input. +func SafeSort(input []float64) int { + nans := 0 + for i := 0; i < len(input); i++ { + if math.IsNaN(input[i]) { + nans++ + } + } + + sort.Float64s(input) + return nans +} + +// GetPercentile computes the percentile cut off for an array of floats +func GetPercentile(input []float64, percentile float64, interpolate bool) float64 { + nans := SafeSort(input) + series := input[nans:] + if len(series) == 0 { + return math.NaN() + } + + fractionalRank := (percentile / 100.0) * (float64(len(series))) + rank := math.Ceil(fractionalRank) + + rankAsInt := int(rank) + + if rankAsInt <= 1 { + return series[0] + } + + percentileResult := series[rankAsInt-1] + + if interpolate { + prevValue := series[rankAsInt-2] + fraction := fractionalRank - (rank - 1) + percentileResult = prevValue + (fraction * (percentileResult - prevValue)) + } + + return percentileResult +} + +// NPercentile returns percentile-percent of each series in the seriesList. +func NPercentile(ctx *Context, in ts.SeriesList, percentile float64, pn PercentileNamer) (ts.SeriesList, error) { + if percentile < 0.0 || percentile > 100.0 { + return ts.SeriesList{}, ErrInvalidPercentile(percentile) + } + results := make([]*ts.Series, 0, in.Len()) + for _, s := range in.Values { + safeValues := s.SafeValues() + if len(safeValues) == 0 { + continue + } + percentileVal := GetPercentile(safeValues, percentile, false) + if !math.IsNaN(percentileVal) { + vals := ts.NewConstantValues(ctx, percentileVal, s.Len(), s.MillisPerStep()) + percentileSeries := ts.NewSeries(ctx, pn(s.Name(), percentile), s.StartTime(), vals) + results = append(results, percentileSeries) + } + } + in.Values = results + return in, nil +} + +// RemoveByPercentile removes all series above or below the given percentile, as +// determined by the PercentileComparator +func RemoveByPercentile( + ctx *Context, + in ts.SeriesList, + percentile float64, + pn PercentileNamer, + tc ThresholdComparator, +) (ts.SeriesList, error) { + results := make([]*ts.Series, 0, in.Len()) + for _, series := range in.Values { + single := ts.SeriesList{Values: []*ts.Series{series}} + percentileSeries, err := NPercentile(ctx, single, percentile, pn) + if err != nil { + return ts.SeriesList{}, err + } + + numSteps := series.Len() + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + if percentileSeries.Len() == 1 { + percentile := percentileSeries.Values[0].ValueAt(0) + for i := 0; i < numSteps; i++ { + v := series.ValueAt(i) + if !tc(v, percentile) { + vals.SetValueAt(i, v) + } + } + } + name := pn(series.Name(), percentile) + newSeries := ts.NewSeries(ctx, name, series.StartTime(), vals) + results = append(results, newSeries) + } + in.Values = results + return in, nil +} diff --git a/src/query/graphite/common/percentiles_test.go b/src/query/graphite/common/percentiles_test.go new file mode 100644 index 0000000000..e42c7375f5 --- /dev/null +++ b/src/query/graphite/common/percentiles_test.go @@ -0,0 +1,114 @@ +// Copyright (c) 2019 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 common + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +type percentileTestParams struct { + interpolate bool + percentile float64 + input []float64 + expected float64 +} + +func TestGetPercentile(t *testing.T) { + tests := []percentileTestParams{ + { + false, + 0, + []float64{1, 2, 3, 4, 5}, + 1, + }, + { + false, + 10, + []float64{1, 2, 3, 4, 5}, + 1, + }, + { + false, + 50, + []float64{1, 2, 3, 4, 5}, + 3, + }, + { + true, + 50, + []float64{1, 2, 3, 4, 5}, + 2.5, + }, + { + false, + 50, + []float64{1, 2, 3, 4, 5, 6}, + 3, + }, + { + true, + 50, + []float64{1, 2, 3, 4, 5, 6}, + 3, + }, + { + false, + 90, + []float64{1, 2, 3, 4, 5}, + 5, + }, + { + false, + 50, + []float64{1}, + 1, + }, + { + false, + 50, + []float64{1, 2}, + 1, + }, + { + true, + 30, + []float64{32, 34, 62, 73, 75}, + 33, + }, + { + true, + 33, + []float64{32, 34, 73, 75}, + 32.64, + }, + } + + for _, test := range tests { + testGetPercentile(t, test) + } +} + +func testGetPercentile(t *testing.T, test percentileTestParams) { + actual := GetPercentile(test.input, test.percentile, test.interpolate) + assert.Equal(t, test.expected, actual) +} diff --git a/src/query/graphite/common/test_util.go b/src/query/graphite/common/test_util.go new file mode 100644 index 0000000000..95138b5364 --- /dev/null +++ b/src/query/graphite/common/test_util.go @@ -0,0 +1,161 @@ +// Copyright (c) 2019 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 common + +import ( + "math" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/storage" + xtest "github.com/m3db/m3/src/query/graphite/testing" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// TestSeries is used to create a tsdb.timeSeries +type TestSeries struct { + Name string + Data []float64 +} + +// NewTestContext creates a new test context. +func NewTestContext() *Context { + now := time.Now() + return NewContext(ContextOptions{Start: now.Add(-time.Hour), End: now}) +} + +// NewTestSeriesValues creates a new ts.Values with given step size and values. +func NewTestSeriesValues(ctx context.Context, millisPerStep int, values []float64) ts.Values { + tsv := ts.NewValues(ctx, millisPerStep, len(values)) + + for i, n := range values { + tsv.SetValueAt(i, n) + } + + return tsv +} + +// NewTestSeriesList creates a test series and values from a set of inputs +func NewTestSeriesList(ctx *Context, start time.Time, inputs []TestSeries, step int) []*ts.Series { + seriesList := make([]*ts.Series, 0, len(inputs)) + + for _, in := range inputs { + series := ts.NewSeries(ctx, in.Name, start, NewTestSeriesValues(ctx, step, in.Data)) + seriesList = append(seriesList, series) + } + + return seriesList +} + +// NewConsolidationTestSeries returns multiple static series for consolidation +func NewConsolidationTestSeries(start, end time.Time, duration time.Duration) (*Context, []*ts.Series) { + ctx := NewContext(ContextOptions{Start: start, End: end}) + + testSeries := []*ts.Series{ + ts.NewSeries(ctx, "a", start, + ts.NewConstantValues(ctx, 10, 6, 10000)), + ts.NewSeries(ctx, "b", start.Add(-duration), + ts.NewConstantValues(ctx, 15, 6, 10000)), + ts.NewSeries(ctx, "c", start.Add(duration), + ts.NewConstantValues(ctx, 17, 6, 10000)), + ts.NewSeries(ctx, "d", start, + ts.NewConstantValues(ctx, 3, 60, 1000)), + } + return ctx, testSeries +} + +// CompareOutputsAndExpected compares the actual output with the expected output. +func CompareOutputsAndExpected(t *testing.T, step int, start time.Time, expected []TestSeries, + actual []*ts.Series) { + require.Equal(t, len(expected), len(actual)) + for i := range expected { + a := actual[i] + require.Equal(t, expected[i].Name, a.Name()) + assert.Equal(t, step, a.MillisPerStep(), a.Name()+": MillisPerStep in expected series do not match MillisPerStep in actual") + assert.Equal(t, start, a.StartTime(), a.Name()+": StartTime in expected series does not match StartTime in actual") + e := expected[i].Data + require.Equal(t, len(e), a.Len(), a.Name()+": length of expected series does not match length of actual") + for step := 0; step < a.Len(); step++ { + v := a.ValueAt(step) + if math.IsNaN(e[step]) { + assert.True(t, math.IsNaN(v), a.Name()+": invalid value for step %d/%d, should be NaN but is %v", step, a.Len(), v) + } else { + xtest.InDeltaWithNaNs(t, e[step], v, 0.0001, a.Name()+": invalid value for %d/%d", step, a.Len()) + } + } + } +} + +// MovingAverageStorage is a special test construct for the moving average function +type MovingAverageStorage struct { + StepMillis int + Bootstrap []float64 + Values []float64 + BootstrapStart time.Time +} + +// FetchByPath builds a new series from the input path +func (s *MovingAverageStorage) FetchByPath( + ctx context.Context, + path string, + opts storage.FetchOptions, +) (*storage.FetchResult, error) { + return s.fetchByIDs(ctx, []string{path}, opts) +} + +// FetchByQuery builds a new series from the input query +func (s *MovingAverageStorage) FetchByQuery( + ctx context.Context, + query string, + opts storage.FetchOptions, +) (*storage.FetchResult, error) { + return s.fetchByIDs(ctx, []string{query}, opts) +} + +// FetchByIDs builds a new series from the input query +func (s *MovingAverageStorage) fetchByIDs( + ctx context.Context, + ids []string, + opts storage.FetchOptions, +) (*storage.FetchResult, error) { + var seriesList []*ts.Series + if s.Bootstrap != nil || s.Values != nil { + var values []float64 + if opts.StartTime.Equal(s.BootstrapStart) { + values = s.Bootstrap + } else { + values = s.Values + } + series := ts.NewSeries(ctx, ids[0], opts.StartTime, NewTestSeriesValues(ctx, s.StepMillis, values)) + seriesList = append(seriesList, series) + } + return storage.NewFetchResult(ctx, seriesList), nil +} + +// Type returns to the dc scope +// func (s *MovingAverageStorage) Type() storage.Type { return storage.TypeLocalDC } + +// Name returns the name of the storage struct +// func (s *MovingAverageStorage) Name() string { return "movingAverageStorage" } diff --git a/src/query/graphite/common/transform.go b/src/query/graphite/common/transform.go new file mode 100644 index 0000000000..b902f5aa70 --- /dev/null +++ b/src/query/graphite/common/transform.go @@ -0,0 +1,312 @@ +// Copyright (c) 2019 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 common + +import ( + "fmt" + "math" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +// TransformFunc is used by Transform to apply a function +// to all values in a series. +type TransformFunc func(float64) float64 + +// TransformFuncFactory creates transformation functions +type TransformFuncFactory func() TransformFunc + +// Transformer transforms a value +type Transformer interface { + // Apply applies the transformation + Apply(value float64) float64 + + // Reset resets the state + Reset() +} + +type statelessTransformer struct { + fn TransformFunc +} + +// NewStatelessTransformer creates a new stateless transformer +func NewStatelessTransformer(fn TransformFunc) Transformer { + return statelessTransformer{fn: fn} +} + +func (t statelessTransformer) Apply(value float64) float64 { + return t.fn(value) +} + +func (t statelessTransformer) Reset() {} + +// MaintainNaNTransformer only applies a given ValueTransformer to +// non-NaN values. +func MaintainNaNTransformer(f TransformFunc) TransformFunc { + return func(v float64) float64 { + if math.IsNaN(v) { + return v + } + return f(v) + } +} + +// Scale multiplies each element of a series list by a given value. +func Scale(scale float64) TransformFunc { + return MaintainNaNTransformer(func(v float64) float64 { + return v * scale + }) +} + +// Offset adds a value to each element of a series list. +func Offset(factor float64) TransformFunc { + return MaintainNaNTransformer(func(v float64) float64 { + return v + factor + }) +} + +// TransformNull transforms all nulls in a series to a value. +func TransformNull(value float64) TransformFunc { + return func(v float64) float64 { + if math.IsNaN(v) { + return value + } + + return v + } +} + +// IsNonNull takes a series or series list and counts up how many non-null values are specified. +// This is useful for understanding which series have data at a given point in time (i.e. to count +// which servers are alive). +func IsNonNull() TransformFunc { + return func(v float64) float64 { + if math.IsNaN(v) { + return 0 + } + + return 1 + } +} + +// PredicateFn is a predicate function. +type PredicateFn func(v float64) bool + +// Filter removes data that does not satisfy a given predicate. +func Filter(fn PredicateFn) TransformFunc { + return MaintainNaNTransformer(func(v float64) float64 { + if !fn(v) { + return math.NaN() + } + + return v + }) +} + +// Logarithm takes one series or a series list, and draws the y-axis in logarithmic format. Only support +// base 10 logarithms. +func Logarithm() TransformFunc { + return func(v float64) float64 { + if !math.IsNaN(v) && v > 0 { + return math.Log10(v) + } + + return math.NaN() + } +} + +// Integral returns a function that accumulates values it has seen +func Integral() TransformFunc { + currentSum := 0.0 + + return func(v float64) float64 { + if !math.IsNaN(v) { + currentSum += v + } else { + return v + } + return currentSum + } +} + +// Derivative returns a function that computes the derivative among the values +// it has seen +func Derivative() TransformFunc { + previousValue := math.NaN() + + return func(v float64) float64 { + var r float64 + if math.IsNaN(v) || math.IsNaN(previousValue) { + previousValue, r = v, math.NaN() + } else { + previousValue, r = v, v-previousValue + } + return r + } +} + +// NonNegativeDerivative returns a function that computes the derivative among the +// values it has seen but ignores datapoints that trend down +func NonNegativeDerivative(maxValue float64) TransformFunc { + previousValue := math.NaN() + + return func(v float64) float64 { + var r float64 + + if math.IsNaN(v) || math.IsNaN(previousValue) { + previousValue, r = v, math.NaN() + } else if difference := v - previousValue; difference >= 0 { + previousValue, r = v, difference + } else if !math.IsNaN(maxValue) && maxValue >= v { + previousValue, r = v, (maxValue-previousValue)+v+1.0 + } else { + previousValue, r = v, math.NaN() + } + return r + } +} + +// Transform applies a specified ValueTransform to all values in each series, renaming +// each series with the given SeriesRenamer. +func Transform(ctx *Context, in ts.SeriesList, t Transformer, renamer SeriesRenamer) (ts.SeriesList, error) { + results := make([]*ts.Series, in.Len()) + + for i, series := range in.Values { + t.Reset() + values := ts.NewValues(ctx, series.MillisPerStep(), series.Len()) + for step := 0; step < series.Len(); step++ { + value := series.ValueAt(step) + values.SetValueAt(step, t.Apply(value)) + } + + results[i] = ts.NewSeries(ctx, renamer(series), series.StartTime(), values) + } + + in.Values = results + return in, nil +} + +// Stdev takes one metric or a wildcard seriesList followed by an integer N. Draw the standard deviation +// of all metrics passed for the past N datapoints. If the ratio of null points in the window is greater than +// windowTolerance, skip the calculation. +func Stdev(ctx *Context, in ts.SeriesList, points int, windowTolerance float64, renamer RenamerWithNumPoints) (ts.SeriesList, error) { + if points <= 0 { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("invalid window size, points=%d", points)) + } + results := make([]*ts.Series, 0, in.Len()) + for _, series := range in.Values { + stdevName := renamer(series, points) + stdevVals := ts.NewValues(ctx, series.MillisPerStep(), series.Len()) + validPoints := 0 + currentSum := 0.0 + currentSumOfSquares := 0.0 + for index := 0; index < series.Len(); index++ { + newValue := series.ValueAt(index) + var bootstrapping bool + var droppedValue float64 + + // Mark whether we've reached our window size, don't drop points out otherwise + if index < points { + bootstrapping = true + droppedValue = math.NaN() + } else { + bootstrapping = false + droppedValue = series.ValueAt(index - points) + } + + // Remove the value that just dropped out of the window + if !bootstrapping && !math.IsNaN(droppedValue) { + validPoints-- + currentSum -= droppedValue + currentSumOfSquares -= droppedValue * droppedValue + } + + // Add in the value that just popped in the window + if !math.IsNaN(newValue) { + validPoints++ + currentSum += newValue + currentSumOfSquares += newValue * newValue + } + + if validPoints > 0 && float64(validPoints)/float64(points) >= windowTolerance { + deviation := math.Sqrt(float64(validPoints)*currentSumOfSquares-currentSum*currentSum) / float64(validPoints) + stdevVals.SetValueAt(index, deviation) + } + } + stdevSeries := ts.NewSeries(ctx, stdevName, series.StartTime(), stdevVals) + results = append(results, stdevSeries) + } + in.Values = results + return in, nil +} + +// RenamerWithNumPoints is a signature for renaming a single series that is passed to Stdev +type RenamerWithNumPoints func(series *ts.Series, points int) string + +// PerSecond computes the derivative between consecutive values in the a time series, taking into +// account the time interval between the values. It skips missing values, and calculates the +// derivative between consecutive non-missing values. +func PerSecond(ctx *Context, in ts.SeriesList, renamer SeriesRenamer) (ts.SeriesList, error) { + results := make([]*ts.Series, 0, in.Len()) + + for _, series := range in.Values { + var ( + vals = ts.NewValues(ctx, series.MillisPerStep(), series.Len()) + prev = math.NaN() + secsPerStep = float64(series.MillisPerStep()) / 1000 + secsSinceLastVal = secsPerStep + ) + + for step := 0; step < series.Len(); step++ { + cur := series.ValueAt(step) + + if math.IsNaN(prev) { + vals.SetValueAt(step, math.NaN()) + prev = cur + continue + } + + if math.IsNaN(cur) { + vals.SetValueAt(step, math.NaN()) + secsSinceLastVal += secsPerStep + continue + } + + diff := cur - prev + + if diff >= 0 { + vals.SetValueAt(step, diff/secsSinceLastVal) + } else { + vals.SetValueAt(step, math.NaN()) + } + + prev = cur + secsSinceLastVal = secsPerStep + } + + s := ts.NewSeries(ctx, renamer(series), series.StartTime(), vals) + results = append(results, s) + } + + in.Values = results + return in, nil +} diff --git a/src/query/graphite/common/transform_test.go b/src/query/graphite/common/transform_test.go new file mode 100644 index 0000000000..c1d782f381 --- /dev/null +++ b/src/query/graphite/common/transform_test.go @@ -0,0 +1,380 @@ +// Copyright (c) 2019 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 common + +import ( + "fmt" + "math" + "testing" + "time" + + xtest "github.com/m3db/m3/src/query/graphite/testing" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func testRenamer(series *ts.Series) string { + return fmt.Sprintf("test %v", series.Name()) +} + +func TestAbsolute(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + var ( + vals = []float64{-2, 0, 42, math.NaN()} + step = 100 + now = time.Now() + ) + input := ts.SeriesList{ + Values: []*ts.Series{ + ts.NewSeries(ctx, "vodka", now, NewTestSeriesValues(ctx, step, vals)), + }, + } + + r, err := Transform(ctx, input, NewStatelessTransformer(math.Abs), testRenamer) + require.NoError(t, err) + + output := r.Values + require.Equal(t, 1, len(output)) + + abs := output[0] + require.Equal(t, len(vals), abs.Len()) + assert.Equal(t, step, abs.MillisPerStep()) + assert.Equal(t, now, abs.StartTime()) + assert.Equal(t, "test vodka", abs.Name()) + + absVals := make([]float64, len(vals)) + for i := 0; i < abs.Len(); i++ { + absVals[i] = abs.ValueAt(i) + } + xtest.Equalish(t, []float64{2, 0, 42, math.NaN()}, absVals) +} + +func TestOffset(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + var ( + tests = []struct { + inputs []float64 + factor float64 + output []float64 + }{ + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, 2.5, + []float64{2.5, 3.5, 4.5, math.NaN(), 5.5}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, -0.5, + []float64{-0.5, 0.5, 1.5, math.NaN(), 2.5}, + }, + } + + startTime = time.Now() + step = 100 + ) + + for _, test := range tests { + input := ts.SeriesList{ + Values: []*ts.Series{ + ts.NewSeries(ctx, "foo", startTime, NewTestSeriesValues(ctx, step, test.inputs)), + }, + } + + r, err := Transform(ctx, input, NewStatelessTransformer(Offset(test.factor)), testRenamer) + require.NoError(t, err) + + output := r.Values + require.EqualValues(t, 1, len(output)) + require.Equal(t, len(test.inputs), output[0].Len()) + + assert.Equal(t, step, output[0].MillisPerStep()) + assert.Equal(t, startTime, output[0].StartTime()) + assert.Equal(t, "test foo", output[0].Name()) + + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + xtest.EqualWithNaNs(t, float64(test.output[step]), float64(v), "invalid value for %d", step) + } + } + +} + +func TestScale(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + var ( + tests = []struct { + inputs []float64 + scale float64 + output []float64 + }{ + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, 2.5, + []float64{0, 2.5, 5.0, math.NaN(), 7.5}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, 0.5, + []float64{0, 0.5, 1.0, math.NaN(), 1.5}, + }, + } + + startTime = time.Now() + step = 100 + ) + + for _, test := range tests { + input := ts.SeriesList{ + Values: []*ts.Series{ + ts.NewSeries(ctx, "foo", startTime, NewTestSeriesValues(ctx, step, test.inputs)), + }, + } + + r, err := Transform(ctx, input, NewStatelessTransformer(Scale(test.scale)), testRenamer) + require.NoError(t, err) + + output := r.Values + require.EqualValues(t, 1, len(output)) + require.Equal(t, len(test.inputs), output[0].Len()) + + assert.EqualValues(t, step, output[0].MillisPerStep()) + assert.Equal(t, startTime, output[0].StartTime()) + assert.Equal(t, "test foo", output[0].Name()) + + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + xtest.EqualWithNaNs(t, float64(test.output[step]), float64(v), "invalid value for %d", step) + } + } +} + +func TestTransformNull(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + var ( + tests = []struct { + inputs []float64 + defaultValue float64 + output []float64 + }{ + { + []float64{0, math.NaN(), 2.0, math.NaN(), 3.0}, 42.5, + []float64{0, 42.5, 2.0, 42.5, 3.0}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, -0.5, + []float64{0, 1.0, 2.0, -0.5, 3.0}, + }, + } + + startTime = time.Now() + step = 100 + ) + + for _, test := range tests { + input := ts.SeriesList{ + Values: []*ts.Series{ + ts.NewSeries(ctx, "foo", startTime, NewTestSeriesValues(ctx, step, test.inputs)), + }, + } + + r, err := Transform(ctx, input, NewStatelessTransformer(TransformNull(test.defaultValue)), testRenamer) + require.NoError(t, err) + + output := r.Values + require.EqualValues(t, 1, len(output)) + require.Equal(t, len(test.inputs), output[0].Len()) + + assert.EqualValues(t, step, output[0].MillisPerStep()) + assert.Equal(t, startTime, output[0].StartTime()) + assert.Equal(t, "test foo", output[0].Name()) + + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + assert.Equal(t, test.output[step], v, "invalid value for %d", step) + } + } +} + +func TestIsNonNull(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + var ( + tests = []struct { + inputs []float64 + output []float64 + }{ + { + []float64{0, math.NaN(), 2.0, math.NaN(), 3.0}, + []float64{1, 0, 1, 0, 1}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, + []float64{1, 1, 1, 0, 1}, + }, + } + + startTime = time.Now() + step = 100 + ) + + for _, test := range tests { + input := ts.SeriesList{ + Values: []*ts.Series{ + ts.NewSeries(ctx, "foo", startTime, NewTestSeriesValues(ctx, step, test.inputs)), + }, + } + + r, err := Transform(ctx, input, NewStatelessTransformer(IsNonNull()), testRenamer) + require.NoError(t, err) + + output := r.Values + require.EqualValues(t, 1, len(output)) + require.Equal(t, len(test.inputs), output[0].Len()) + + assert.EqualValues(t, step, output[0].MillisPerStep()) + assert.Equal(t, startTime, output[0].StartTime()) + assert.Equal(t, "test foo", output[0].Name()) + + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + assert.Equal(t, test.output[step], v, "invalid value for %d", step) + } + } +} + +func TestStdev(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + inputs := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + { + "foo", + startTime, + stepSize, + []float64{1.0, 2.0, 3.0, 4.0, nan, nan, nan, 5.0, 6.0, nan, nan}, + }, + } + + inputSeries := make([]*ts.Series, 0, len(inputs)) + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + inputSeries = append(inputSeries, series) + } + expected := []TestSeries{ + TestSeries{Name: "foo | stddev 3", Data: []float64{0.0, 0.5, 0.8165, 0.8165, 0.5, 0.0, nan, 0.0, 0.5, 0.5, 0.0}}, + } + input := ts.SeriesList{Values: inputSeries} + results, err := Stdev(ctx, input, 3, 0.1, func(series *ts.Series, points int) string { + return fmt.Sprintf("%s | stddev %d", series.Name(), points) + }) + require.Nil(t, err) + CompareOutputsAndExpected(t, stepSize, startTime, expected, results.Values) +} + +func TestPerSecond(t *testing.T) { + ctx := NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 1000 + inputs := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + { + "foo", + startTime, + stepSize, + []float64{1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 9.0, 10.0}, + }, + { + "foo", + startTime, + stepSize, + []float64{1.0, 2.0, 4.0, 7.0, 11.0, 16.0, 22.0, 29.0, 37.0, 46.0}, + }, + { + "foo", + startTime, + stepSize, + []float64{1.0, 2.0, 3.0, 4.0, 5.0, 1.0, 2.0, 3.0, 4.0, 5.0}, + }, + { + "foo", + startTime, + stepSize, + []float64{nan, nan, nan, 4.0, 5.0, 1.0, 2.0, 3.0, 4.0, 5.0}, + }, + { + "foo", + startTime, + stepSize, + []float64{1.0, 2.0, 3.0, nan, nan, nan, 7.0, 8.0, 9.0, 10.0}, + }, + } + + inputSeries := make([]*ts.Series, 0, len(inputs)) + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + inputSeries = append(inputSeries, series) + } + expected := []TestSeries{ + TestSeries{Name: "foo | perSecond", Data: []float64{nan, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0}}, + TestSeries{Name: "foo | perSecond", Data: []float64{nan, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 9.0}}, + TestSeries{Name: "foo | perSecond", Data: []float64{nan, 1.0, 1.0, 1.0, 1.0, nan, 1.0, 1.0, 1.0, 1.0}}, + TestSeries{Name: "foo | perSecond", Data: []float64{nan, nan, nan, nan, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0}}, + TestSeries{Name: "foo | perSecond", Data: []float64{nan, 1.0, 1.0, nan, nan, nan, 1.0, 1.0, 1.0, 1.0}}, + } + input := ts.SeriesList{Values: inputSeries} + results, err := PerSecond(ctx, input, func(series *ts.Series) string { + return fmt.Sprintf("%s | perSecond", series.Name()) + }) + require.Nil(t, err) + CompareOutputsAndExpected(t, stepSize, startTime, expected, results.Values) +} diff --git a/src/query/graphite/context/context.go b/src/query/graphite/context/context.go new file mode 100644 index 0000000000..1904a8fad8 --- /dev/null +++ b/src/query/graphite/context/context.go @@ -0,0 +1,164 @@ +// Copyright (c) 2019 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 context + +import ( + ctx "context" + "sync" +) + +// Closer is an interface implemented by objects that should be closed +// when a context completes +type Closer interface { + Close() error +} + +// Context is an execution context. A Context holds deadlines and request +// scoped values. Contexts should be scoped to a request; +// created when the request begins and closed when the request completes. +// Contexts are safe for use by multiple goroutines, but should not span +// multiple requests. +type Context interface { + Closer + + // SetRequestContext sets the given context as the request context for this + // execution context. This is used for calls to the m3 storage wrapper. + SetRequestContext(ctx.Context) + + // RequestContext will provide the wrapped request context. Used for calls + // to m3 storage wrapper. + RequestContext() ctx.Context + + // RegisterCloser registers an object that should be closed when this + // context is closed. Can be used to cleanup per-request objects. + RegisterCloser(closer Closer) + + // AddAsyncTask allows asynchronous tasks to be enqueued that will + // ensure this context does not call its registered closers until + // the tasks are all complete + AddAsyncTasks(count int) + + // DoneAsyncTask signals that an asynchronous task is complete, when + // all asynchronous tasks complete if the context has been closed and + // avoided calling its registered closers it will finally call them + DoneAsyncTask() +} + +// New creates a new context +func New() Context { + return &context{} +} + +type contextStatus int + +const ( + contextStatusOpen contextStatus = iota + contextStatusClosed +) + +type context struct { + sync.RWMutex + closers []Closer + status contextStatus + asyncTasks int + reqCtx ctx.Context +} + +// Close closes the context +func (c *context) Close() error { + finalize := false + + c.Lock() + if c.status == contextStatusOpen { + if c.asyncTasks == 0 { + finalize = true + } + c.status = contextStatusClosed + } + c.Unlock() + + if finalize { + return c.callClosers() + } + return nil +} + +// SetRequestContext sets the given context as the request context for this +// execution context. This is used for calls to the m3 storage wrapper. +func (c *context) SetRequestContext(reqCtx ctx.Context) { + c.Lock() + c.reqCtx = reqCtx + c.Unlock() +} + +// RequestContext will provide the wrapped request context. Used for calls +// to m3 storage wrapper. +func (c *context) RequestContext() ctx.Context { + c.RLock() + r := c.reqCtx + c.RUnlock() + return r +} + +// RegisterCloser registers a new Closer with the context +func (c *context) RegisterCloser(closer Closer) { + c.Lock() + c.closers = append(c.closers, closer) + c.Unlock() +} + +// AddAsyncTasks adds tracked asynchronous task(s) +func (c *context) AddAsyncTasks(count int) { + c.Lock() + c.asyncTasks += count + c.Unlock() +} + +// DoneAsyncTask marks a single tracked asynchronous task complete +func (c *context) DoneAsyncTask() { + finalize := false + + c.Lock() + c.asyncTasks-- + if c.asyncTasks == 0 && c.status == contextStatusClosed { + finalize = true + } + c.Unlock() + + if finalize { + c.callClosers() + } +} + +func (c *context) callClosers() error { + var firstErr error + c.RLock() + for _, closer := range c.closers { + if err := closer.Close(); err != nil { + //FIXME: log.Errorf("could not close %v: %v", closer, err) + if firstErr == nil { + firstErr = err + } + } + } + c.RUnlock() + return firstErr +} diff --git a/src/query/graphite/errors/errors.go b/src/query/graphite/errors/errors.go new file mode 100644 index 0000000000..173cce27ae --- /dev/null +++ b/src/query/graphite/errors/errors.go @@ -0,0 +1,99 @@ +// Copyright (c) 2019 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 errors + +import ( + errs "errors" +) + +// New returns an error that formats as the given text +func New(name string) error { + return errs.New(name) +} + +type containedError struct { + inner error +} + +type containedErr interface { + innerError() error +} + +// InnerError returns the packaged inner error if this is an error that contains another +func InnerError(err error) error { + contained, ok := err.(containedErr) + if !ok { + return nil + } + return contained.innerError() +} + +type invalidParamsError struct { + containedError +} + +// NewInvalidParamsError creates a new invalid params error +func NewInvalidParamsError(inner error) error { + return invalidParamsError{containedError{inner}} +} + +func (e invalidParamsError) Error() string { + return e.inner.Error() +} + +func (e invalidParamsError) innerError() error { + return e.inner +} + +// IsInvalidParams returns true if this is an invalid params error +func IsInvalidParams(err error) bool { + return GetInnerInvalidParamsError(err) != nil +} + +// GetInnerInvalidParamsError returns an inner invalid params error +// if contained by this error, nil otherwise +func GetInnerInvalidParamsError(err error) error { + for err != nil { + if _, ok := err.(invalidParamsError); ok { + return InnerError(err) + } + err = InnerError(err) + } + return nil +} + +type renamedError struct { + containedError + renamed error +} + +// NewRenamedError returns a new error that packages an inner error with a renamed error +func NewRenamedError(inner, renamed error) error { + return renamedError{containedError{inner}, renamed} +} + +func (e renamedError) Error() string { + return e.renamed.Error() +} + +func (e renamedError) innerError() error { + return e.inner +} diff --git a/src/query/graphite/errors/errors_test.go b/src/query/graphite/errors/errors_test.go new file mode 100644 index 0000000000..919bbde556 --- /dev/null +++ b/src/query/graphite/errors/errors_test.go @@ -0,0 +1,39 @@ +// Copyright (c) 2019 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 errors + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestInvalidParamsError(t *testing.T) { + var ( + innerErr = errors.New("inner") + err = NewInvalidParamsError(innerErr) + ) + + assert.Error(t, err) + assert.True(t, IsInvalidParams(err)) + assert.Equal(t, innerErr, GetInnerInvalidParamsError(err)) +} diff --git a/src/query/graphite/graphite/glob.go b/src/query/graphite/graphite/glob.go new file mode 100644 index 0000000000..00853e1a69 --- /dev/null +++ b/src/query/graphite/graphite/glob.go @@ -0,0 +1,188 @@ +// Copyright (c) 2019 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 graphite + +import ( + "bytes" + "fmt" + "strings" + + "github.com/m3db/m3/src/query/graphite/errors" +) + +const ( + // ValidIdentifierRunes contains all the runes allowed in a graphite identifier + ValidIdentifierRunes = "ABCDEFGHIJKLMNOPQRSTUVWXYZ" + + "abcdefghijklmnopqrstuvwxyz" + + "0123456789" + + "$-_'|<>%#/" +) + +var ( + // ErrNotPattern signifies that the provided string is not a glob pattern + ErrNotPattern = errors.NewInvalidParamsError(fmt.Errorf("not a pattern")) +) + +// GlobToRegexPattern converts a graphite-style glob into a regex pattern +func GlobToRegexPattern(glob string) (string, error) { + return globToRegexPattern(glob, GlobOptions{}) +} + +// ExtendedGlobToRegexPattern converts a graphite-style glob into a regex pattern +// with extended options +func ExtendedGlobToRegexPattern(glob string, opts GlobOptions) (string, error) { + return globToRegexPattern(glob, opts) +} + +// GlobOptions allows for matching everything +type GlobOptions struct { + // AllowMatchAll allows for matching all text + // including hierarchy separators with "**" + AllowMatchAll bool `yaml:"allowMatchAll"` +} + +type pattern struct { + buff bytes.Buffer + eval rune + lastWriteLen int +} + +func (p *pattern) String() string { + return p.buff.String() +} + +func (p *pattern) Evaluate(r rune) { + p.eval = r +} + +func (p *pattern) LastEvaluate() rune { + return p.eval +} + +func (p *pattern) WriteRune(r rune) { + p.buff.WriteRune(r) + p.lastWriteLen = 1 +} + +func (p *pattern) WriteString(str string) { + p.buff.WriteString(str) + p.lastWriteLen = len(str) +} + +func (p *pattern) UnwriteLast() { + p.buff.Truncate(p.buff.Len() - p.lastWriteLen) + p.lastWriteLen = 0 +} + +func globToRegexPattern(glob string, opts GlobOptions) (string, error) { + var ( + pattern pattern + escaping = false + regexed = false + ) + + groupStartStack := []rune{rune(0)} // rune(0) indicates pattern is not in a group + for i, r := range glob { + prevEval := pattern.LastEvaluate() + pattern.Evaluate(r) + + if escaping { + pattern.WriteRune(r) + escaping = false + continue + } + + switch r { + case '\\': + escaping = true + pattern.WriteRune('\\') + case '.': + // Match hierarchy separator + pattern.WriteString("\\.+") + regexed = true + case '?': + // Match anything except the hierarchy separator + pattern.WriteString("[^\\.]") + regexed = true + case '*': + if opts.AllowMatchAll && prevEval == '*' { + pattern.UnwriteLast() + pattern.WriteString(".*") + regexed = true + } else { + // Match everything up to the next hierarchy separator + pattern.WriteString("[^\\.]*") + regexed = true + } + case '{': + // Begin non-capturing group + pattern.WriteString("(") + groupStartStack = append(groupStartStack, r) + regexed = true + case '}': + // End non-capturing group + priorGroupStart := groupStartStack[len(groupStartStack)-1] + if priorGroupStart != '{' { + return "", errors.NewInvalidParamsError(fmt.Errorf("invalid '}' at %d, no prior for '{' in %s", i, glob)) + } + + pattern.WriteRune(')') + groupStartStack = groupStartStack[:len(groupStartStack)-1] + case '[': + // Begin character range + pattern.WriteRune('[') + groupStartStack = append(groupStartStack, r) + regexed = true + case ']': + // End character range + priorGroupStart := groupStartStack[len(groupStartStack)-1] + if priorGroupStart != '[' { + return "", errors.NewInvalidParamsError(fmt.Errorf("invalid ']' at %d, no prior for '[' in %s", i, glob)) + } + + pattern.WriteRune(']') + groupStartStack = groupStartStack[:len(groupStartStack)-1] + case '<', '>', '\'', '$': + pattern.WriteRune('\\') + pattern.WriteRune(r) + case ',': + // Commas are part of the pattern if they appear in a group + if groupStartStack[len(groupStartStack)-1] == '{' { + pattern.WriteRune('|') + } else { + return "", errors.NewInvalidParamsError(fmt.Errorf("invalid ',' outside of matching group at pos %d in %s", i, glob)) + } + default: + if !strings.ContainsRune(ValidIdentifierRunes, r) { + return "", errors.NewInvalidParamsError(fmt.Errorf("invalid character %c at pos %d in %s", r, i, glob)) + } + pattern.WriteRune(r) + } + } + + if len(groupStartStack) > 1 { + return "", errors.NewInvalidParamsError(fmt.Errorf("unbalanced '%c' in %s", groupStartStack[len(groupStartStack)-1], glob)) + } else if !regexed { + return "", ErrNotPattern + } + + return pattern.String(), nil +} diff --git a/src/query/graphite/graphite/glob_test.go b/src/query/graphite/graphite/glob_test.go new file mode 100644 index 0000000000..195bf05555 --- /dev/null +++ b/src/query/graphite/graphite/glob_test.go @@ -0,0 +1,109 @@ +// Copyright (c) 2019 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 graphite + +import ( + "fmt" + "regexp" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestGlobToRegexPattern(t *testing.T) { + tests := []struct { + glob string + regex string + }{ + {"foo\\+bar.'baz<1001>'.qux", "foo\\+bar\\.+\\'baz\\<1001\\>\\'\\.+qux"}, + {"foo.host.me{1,2,3}.*", "foo\\.+host\\.+me(1|2|3)\\.+[^\\.]*"}, + {"bar.zed.whatever[0-9].*.*.bar", "bar\\.+zed\\.+whatever[0-9]\\.+[^\\.]*\\.+[^\\.]*\\.+bar"}, + {"foo{0[3-9],1[0-9],20}", "foo(0[3-9]|1[0-9]|20)"}, + } + + for _, test := range tests { + pattern, err := GlobToRegexPattern(test.glob) + require.NoError(t, err) + assert.Equal(t, test.regex, pattern, "bad pattern for %s", test.glob) + } +} + +func TestGlobToRegexPatternErrors(t *testing.T) { + tests := []struct { + glob string + err string + }{ + {"foo.host{1,2", "unbalanced '{' in foo.host{1,2"}, + {"foo.host{1,2]", "invalid ']' at 12, no prior for '[' in foo.host{1,2]"}, + {"foo.,", "invalid ',' outside of matching group at pos 4 in foo.,"}, + {"foo.host{a[0-}", "invalid '}' at 13, no prior for '{' in foo.host{a[0-}"}, + } + + for _, test := range tests { + _, err := GlobToRegexPattern(test.glob) + require.Error(t, err) + assert.Equal(t, test.err, err.Error(), "invalid error for %s", test.glob) + } +} + +func TestCompileGlob(t *testing.T) { + tests := []struct { + glob string + match bool + toMatch []string + }{ + {"foo.bar.timers.baz??-bar.qux.query.count", true, + []string{ + "foo.bar.timers.baz01-bar.qux.query.count", + "foo.bar.timers.baz24-bar.qux.query.count"}}, + {"foo.bar.timers.baz??-bar.qux.query.count", false, + []string{ + "foo.bar.timers.baz-bar.qux.query.count", + "foo.bar.timers.baz.0-bar.qux.query.count", + "foo.bar.timers.baz021-bar.qux.query.count", + "foo.bar.timers.baz991-bar.qux.query.count"}}, + {"foo.host{1,2}.*", true, + []string{"foo.host1.zed", "foo.host2.whatever"}}, + {"foo.*.zed.*", true, + []string{"foo.bar.zed.eq", "foo.zed.zed.zed"}}, + {"foo.*.zed.*", false, + []string{"bar.bar.zed.zed", "foo.bar.zed", "foo.bar.zed.eq.monk"}}, + {"foo.host{1,2}.zed", true, + []string{"foo.host1.zed", "foo.host2.zed"}}, + {"foo.host{1,2}.zed", false, + []string{"foo.host3.zed", "foo.hostA.zed", "blad.host1.zed", "foo.host1.zed.z"}}, + {"optic{0[3-9],1[0-9],20}", true, + []string{"optic03", "optic10", "optic20"}}, + {"optic{0[3-9],1[0-9],20}", false, + []string{"optic01", "optic21", "optic201", "optic031"}}, + } + + for _, test := range tests { + rePattern, err := GlobToRegexPattern(test.glob) + require.NoError(t, err) + re := regexp.MustCompile(fmt.Sprintf("^%s$", rePattern)) + for _, s := range test.toMatch { + matched := re.MatchString(s) + assert.Equal(t, test.match, matched, "incorrect match between %s and %s", test.glob, s) + } + } +} diff --git a/src/query/graphite/graphite/graphite_types.go b/src/query/graphite/graphite/graphite_types.go new file mode 100644 index 0000000000..ab7bb8bf9f --- /dev/null +++ b/src/query/graphite/graphite/graphite_types.go @@ -0,0 +1,263 @@ +// Copyright (c) 2019 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 graphite + +import ( + "bytes" + "encoding/json" + "fmt" + "math" + "net/http" + "strconv" + "time" + + "github.com/hydrogen18/stalecucumber" +) + +// MIMETypeApplicationPickle defines the MIME type for application/pickle content +const MIMETypeApplicationPickle = "application/pickle" + +// A Timestamp is a time.Time that knows how to marshal and unmarshal +// itself as Graphite expects (as seconds since Unix epoch) +type Timestamp time.Time + +// MarshalJSON marshals the timestamp as JSON +func (t Timestamp) MarshalJSON() ([]byte, error) { + s := strconv.FormatInt(time.Time(t).Unix(), 10) + return []byte(s), nil +} + +// UnmarshalJSON unmarshals the timestamp from JSON +func (t *Timestamp) UnmarshalJSON(data []byte) error { + n, err := strconv.ParseInt(string(data), 10, 64) + if err != nil { + return err + } + + *t = Timestamp(time.Unix(n, 0).UTC()) + return nil +} + +// A Datavalue is a float64 which knows how to marshal and unmarshal +// itself as Graphite expects (NaNs becomes nulls) +type Datavalue float64 + +// MarshalJSON marshals the value as JSON, writing NaNs as nulls +func (v Datavalue) MarshalJSON() ([]byte, error) { + if math.IsNaN(float64(v)) { + return []byte("null"), nil + } + + s := strconv.FormatFloat(float64(v), 'f', -1, 64) + return []byte(s), nil +} + +// UnmarshalJSON unmarshals the value as JSON, converting nulls into NaNs +func (v *Datavalue) UnmarshalJSON(data []byte) error { + s := string(data) + if s == "null" { + *v = Datavalue(math.NaN()) + return nil + } + + n, err := strconv.ParseFloat(s, 64) + *v = Datavalue(n) + return err +} + +// RenderDatapoints are the set of datapoints returned from Graphite rendering +type RenderDatapoints [][]interface{} + +// Add adds a new datapoint to the set of datapoints +func (dp *RenderDatapoints) Add(timestamp time.Time, value float64) { + *dp = append(*dp, []interface{}{Datavalue(value), Timestamp(timestamp)}) +} + +// Get returns the timestamp and value at the given index +func (dp RenderDatapoints) Get(i int) (time.Time, float64) { + value := math.NaN() + if dp[i][0] != nil { + value = dp[i][0].(float64) + } + + switch timestamp := dp[i][1].(type) { + case float64: + return time.Unix(int64(timestamp), 0).UTC(), value + case int: + return time.Unix(int64(timestamp), 0).UTC(), value + case time.Time: + return timestamp, value + default: + panic(fmt.Sprintf("unsupported timestamp type")) + } +} + +// A RenderTarget is the result of rendering a given target +type RenderTarget struct { + Target string `json:"target"` + Datapoints RenderDatapoints `json:"datapoints"` +} + +// RenderResults are the results from a render API call +type RenderResults []RenderTarget + +// A Datapoint is a Timestamp/Value pair representing a single value in a +// target +type Datapoint struct { + Timestamp Timestamp `json:"t"` + Value Datavalue `json:"v"` +} + +// Results are a map of graphite target names to their corresponding datapoints +type Results map[string][]Datapoint + +// RenderResultsPickle is an alternate form of graphite result, consisting of a +// start time, an end time, a step size (in seconds), and values for each step. +// Steps that do not have a value will be NaN +type RenderResultsPickle struct { + Name string `pickle:"name"` + Start uint32 `pickle:"start"` + End uint32 `pickle:"end"` + Step uint32 `pickle:"step"` + Values []interface{} `pickle:"values"` // value can be nil (python 'None') +} + +// Len returns the number of results +func (p RenderResultsPickle) Len() int { return len(p.Values) } + +// ValueAt returns the value at the given step +func (p RenderResultsPickle) ValueAt(n int) float64 { + if p.Values[n] == nil { + return math.NaN() + } + + return p.Values[n].(float64) +} + +// Get returns the timestamp and value at the given index +func (p RenderResultsPickle) Get(i int) (time.Time, float64) { + value := math.NaN() + if p.Values[i] != nil { + value = p.Values[i].(float64) + } + + timestamp := time.Unix(int64(p.Start)+int64(p.Step*uint32(i)), 0).UTC() + return timestamp, value +} + +// ParseRenderResultsPickle parses a byte stream containing a pickle render response +func ParseRenderResultsPickle(b []byte) ([]RenderResultsPickle, error) { + r := bytes.NewReader(b) + + var pickleResults []RenderResultsPickle + if err := stalecucumber.UnpackInto(&pickleResults).From(stalecucumber.Unpickle(r)); err != nil { + return nil, err + } + + //convert stalecucumber.PickleNone to nil + for _, r := range pickleResults { + for i, v := range r.Values { + _, ok := v.(stalecucumber.PickleNone) + if ok { + r.Values[i] = nil + } + } + } + + return pickleResults, nil +} + +// ParseJSONResponse takes a byteBuffer and returns Results +func ParseJSONResponse(b []byte) (Results, error) { + var jsonResults []jsonResult + if err := json.Unmarshal(b, &jsonResults); err != nil { + return nil, err + } + + results := make(Results, len(jsonResults)) + for _, jsonResult := range jsonResults { + datapoints := make([]Datapoint, 0, len(jsonResult.Datapoints)) + for _, jsonPoint := range jsonResult.Datapoints { + if jsonPoint[0] == nil { + jsonPoint[0] = math.NaN() + } + datapoints = append(datapoints, Datapoint{ + Timestamp: Timestamp(time.Unix(int64(jsonPoint[1].(float64)), 0)), + Value: Datavalue(jsonPoint[0].(float64)), + }) + } + + results[jsonResult.Target] = datapoints + } + + return results, nil +} + +type jsonResult struct { + Target string `json:"target"` + Datapoints [][]interface{} `json:"datapoints"` +} + +// RespondWithPickle sends a python pickle response +func RespondWithPickle(w http.ResponseWriter, data interface{}) error { + w.Header().Add("Content-Type", MIMETypeApplicationPickle) + var buf bytes.Buffer + _, err := stalecucumber.NewPickler(&buf).Pickle(data) + if err != nil { + return err + } + + _, err = w.Write(buf.Bytes()) + return err +} + +// MetricsPathMetadata is an internal element of graphite's "completer" format +// for /metrics/find results. sample: {"is_leaf": "1", "path": +// "servers.foo-bar.baz.qux_qaz", "name": +// "qux_qaz"} +type MetricsPathMetadata struct { + Path string `json:"path"` + Name string `json:"name"` + IsLeaf int `json:"is_leaf,string"` // UGLY(jayp): should be a bool, int due to encoding/json +} + +// FindResultsPickle is graphite's pickle format for /metrics/find results +type FindResultsPickle struct { + Path string `pickle:"path" json:"path"` + IsLeaf bool `pickle:"is_leaf" json:"is_leaf"` +} + +// FindResultsCompleterJSON is graphite's "completer" format for /metrics/find +// results sample: {"metrics": [...]} +type FindResultsCompleterJSON struct { + Metrics []MetricsPathMetadata `json:"metrics"` +} + +// FindResultsTreeJSON is graphite's "treeJSON" format for /metrics/find +// results. sample: {"text": "quz", "expandable": 1, "leaf": 0, "id": +// "foo.bar-baz.qux.quz", "allowChildren": 1} +type FindResultsTreeJSON struct { + ID string `json:"id"` // =path + Text string `json:"text"` // =name + Leaf int `json:"leaf"` // =isLeaf + Expandable int `json:"expandable"` // =!isLeaf + AllowChildren int `json:"allowChildren"` // =!isLeaf +} diff --git a/src/query/graphite/graphite/graphite_types_test.go b/src/query/graphite/graphite/graphite_types_test.go new file mode 100644 index 0000000000..6ebf6f64c5 --- /dev/null +++ b/src/query/graphite/graphite/graphite_types_test.go @@ -0,0 +1,170 @@ +// Copyright (c) 2019 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 graphite + +import ( + "bytes" + "encoding/json" + "io/ioutil" + "math" + "testing" + "time" + + xtest "github.com/m3db/m3/src/query/graphite/testing" + + "github.com/hydrogen18/stalecucumber" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestMarshalRenderResults(t *testing.T) { + expectedJSON := "[{\"target\":\"foo.bar\"," + + "\"datapoints\":[[100,1431470141],[null,1431470151],[3.1456,1431470161]]}]" + + tm := time.Date(2015, 5, 12, 22, 35, 41, 0, time.UTC) + + target := RenderTarget{ + Target: "foo.bar", + } + + target.Datapoints.Add(tm, 100) + target.Datapoints.Add(tm.Add(10*time.Second), math.NaN()) + target.Datapoints.Add(tm.Add(20*time.Second), 3.1456) + results := RenderResults{target} + + r, err := json.Marshal(results) + require.Nil(t, err) + assert.Equal(t, expectedJSON, string(r)) + + var parsed RenderResults + err = json.Unmarshal(r, &parsed) + require.Nil(t, err) + + parsedDatapoints := parsed[0].Datapoints + timestamp, value := parsedDatapoints.Get(0) + assert.Equal(t, tm, timestamp) + assert.Equal(t, float64(100), value) + + timestamp, value = parsedDatapoints.Get(1) + assert.Equal(t, tm.Add(10*time.Second), timestamp) + xtest.EqualWithNaNs(t, math.NaN(), value) + + timestamp, value = parsedDatapoints.Get(2) + assert.Equal(t, tm.Add(20*time.Second), timestamp) + assert.Equal(t, float64(3.1456), value) +} + +func TestDatapointAccess(t *testing.T) { + data, err := ioutil.ReadFile("testdata/no-results.json") + require.Nil(t, err) + + var results RenderResults + err = json.Unmarshal(data, &results) + require.Nil(t, err) + require.Equal(t, 1, len(results)) + + require.Equal(t, "the.quick.brown.fox.jumped.over-the.lazy_dog.once.again.p50", results[0].Target) + require.Equal(t, 360, len(results[0].Datapoints)) + + datapoints := results[0].Datapoints + for i := range datapoints { + _, value := datapoints.Get(i) + xtest.EqualWithNaNs(t, math.NaN(), value, "invalid value for %d", i) + } +} + +func TestMarshalUnmarshalJSONResults(t *testing.T) { + expectedJSON := "{\"foo.bar\":[" + + "{\"t\":1431470141,\"v\":100}," + + "{\"t\":1431470151,\"v\":null}," + + "{\"t\":1431470161,\"v\":3.1456}]}" + + tm := time.Date(2015, 5, 12, 22, 35, 41, 0, time.UTC) + results := Results{ + "foo.bar": { + {Timestamp(tm), Datavalue(100)}, + {Timestamp(tm.Add(10 * time.Second)), Datavalue(math.NaN())}, + {Timestamp(tm.Add(20 * time.Second)), Datavalue(3.1456)}, + }, + } + + r, err := json.Marshal(results) + require.Nil(t, err) + assert.Equal(t, expectedJSON, string(r)) + + var unmarshalled Results + require.Nil(t, json.Unmarshal(r, &unmarshalled)) + xtest.Equalish(t, results, unmarshalled) +} + +func TestPickleValueAccess(t *testing.T) { + now := time.Now().Truncate(time.Second).UTC() + r := RenderResultsPickle{ + Name: "foo.bar.baz space", + Start: uint32(now.Unix()), + End: uint32(now.Add(time.Minute).Unix()), + Step: 20, + Values: []interface{}{1.01, nil, -2.02}, + } + + assert.Equal(t, 3, r.Len()) + assert.Equal(t, 1.01, r.ValueAt(0)) + xtest.Equalish(t, math.NaN(), r.ValueAt(1)) + assert.Equal(t, -2.02, r.ValueAt(2)) + + timestamp, n := r.Get(0) + assert.Equal(t, now, timestamp) + assert.Equal(t, 1.01, n) + + timestamp, n = r.Get(1) + xtest.Equalish(t, now.Add(time.Second*20), timestamp) + xtest.EqualWithNaNs(t, math.NaN(), n) + + timestamp, n = r.Get(2) + assert.Equal(t, now.Add(time.Second*40), timestamp) + assert.Equal(t, -2.02, n) +} + +func TestMarshalUnmarshalPickleResults(t *testing.T) { + + f1 := 1.01 + f2 := -2.02 + f3 := math.NaN() // should really become nil + + in := []RenderResultsPickle{ + RenderResultsPickle{ + Name: "foo.bar.baz space", + Start: 1234, + End: 4321, + Step: 1000, + Values: []interface{}{f1, f2, f3, nil}, + }, + } + + buf := new(bytes.Buffer) + _, err := stalecucumber.NewPickler(buf).Pickle(in) + assert.Nil(t, err, "Unable to pickle data") + + out, err := ParseRenderResultsPickle(buf.Bytes()) + assert.Nil(t, err, "Unable to unpickle data") + + xtest.Equalish(t, in, out) +} diff --git a/src/query/graphite/graphite/identify.go b/src/query/graphite/graphite/identify.go new file mode 100644 index 0000000000..917ddd2947 --- /dev/null +++ b/src/query/graphite/graphite/identify.go @@ -0,0 +1,86 @@ +// Copyright (c) 2019 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 graphite + +// DropLastMetricPart returns the metric string without the last segment. +func DropLastMetricPart(metric string) string { + // read string in reverse until encountering a delimiter + for i := len(metric) - 1; i >= 0; i-- { + if metric[i] == '.' { + return metric[:i] + } + } + + return metric[:0] +} + +// CountMetricParts counts the number of segments in the given metric string. +func CountMetricParts(metric string) int { + return countMetricPartsWithDelimiter(metric, '.') +} + +func countMetricPartsWithDelimiter(metric string, delim byte) int { + if len(metric) == 0 { + return 0 + } + + count := 1 + for i := 0; i < len(metric); i++ { + if metric[i] == delim { + count++ + } + } + + return count +} + +// ExtractNthMetricPart returns the nth part of the metric string. Index starts from 0 +// and assumes metrics are delimited by '.'. If n is negative or bigger than the number +// of parts, returns an empty string. +func ExtractNthMetricPart(metric string, n int) string { + return ExtractNthStringPart(metric, n, '.') +} + +// ExtractNthStringPart returns the nth part of the metric string. Index starts from 0. +// If n is negative or bigger than the number of parts, returns an empty string. +func ExtractNthStringPart(target string, n int, delim rune) string { + if n < 0 { + return "" + } + + leftSide := 0 + delimsToGo := n + 1 + for i := 0; i < len(target); i++ { + if target[i] == byte(delim) { + delimsToGo-- + if delimsToGo == 0 { + return target[leftSide:i] + } + leftSide = i + 1 + } + } + + if delimsToGo > 1 { + return "" + } + + return target[leftSide:] +} diff --git a/src/query/graphite/graphite/identify_test.go b/src/query/graphite/graphite/identify_test.go new file mode 100644 index 0000000000..445347a24a --- /dev/null +++ b/src/query/graphite/graphite/identify_test.go @@ -0,0 +1,72 @@ +// Copyright (c) 2019 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 graphite + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +const ( + statsdStat = "foo.bar.baz.qux.quz" + malformedStat = "foobarbazquxquz" + dashedString = "some-other-delimiter" +) + +func TestExtractNthEmptyMetric(t *testing.T) { + assert.Equal(t, "", ExtractNthMetricPart("", 0)) + assert.Equal(t, 0, CountMetricParts("")) +} + +func TestExtractNthMetricPartNoDots(t *testing.T) { + nodots := "foobarbazquxquz" + assert.Equal(t, nodots, ExtractNthMetricPart(malformedStat, 0)) + assert.Equal(t, 1, CountMetricParts(malformedStat)) +} + +func TestExtractNthMetricPartStandardCase(t *testing.T) { + assert.Equal(t, "foo", ExtractNthMetricPart(statsdStat, 0)) + assert.Equal(t, "bar", ExtractNthMetricPart(statsdStat, 1)) + assert.Equal(t, "quz", ExtractNthMetricPart(statsdStat, 4)) + assert.Equal(t, 5, CountMetricParts(statsdStat)) +} + +func TestExtractNthMetricPartPastEnd(t *testing.T) { + assert.Equal(t, "", ExtractNthMetricPart(statsdStat, 10)) +} + +func TestExtractNthMetricPartNegativeN(t *testing.T) { + assert.Equal(t, "", ExtractNthMetricPart(statsdStat, -2)) +} + +func TestExtractNthStringPart(t *testing.T) { + assert.Equal(t, "other", ExtractNthStringPart(dashedString, 1, '-')) + assert.Equal(t, 3, countMetricPartsWithDelimiter(dashedString, '-')) +} + +func TestDropLastMetricPart(t *testing.T) { + assert.Equal(t, "", DropLastMetricPart("")) + assert.Equal(t, "", DropLastMetricPart("abc")) + assert.Equal(t, "abc", DropLastMetricPart("abc.def")) + assert.Equal(t, "abc.def.ghi", DropLastMetricPart("abc.def.ghi.jkl")) + assert.Equal(t, "abc.def.ghi", DropLastMetricPart("abc.def.ghi.")) +} diff --git a/src/query/graphite/graphite/tags.go b/src/query/graphite/graphite/tags.go new file mode 100644 index 0000000000..6333e04677 --- /dev/null +++ b/src/query/graphite/graphite/tags.go @@ -0,0 +1,64 @@ +// Copyright (c) 2019 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 graphite + +import "fmt" + +const ( + // graphiteFormat is the format for graphite metric tag names, which will be + // represented as tag/value pairs in M3. + // NB: stats.gauges.donkey.kong.barrels would become the following tag set: + // {__g0__: stats} + // {__g1__: gauges} + // {__g2__: donkey} + // {__g3__: kong} + // {__g4__: barrels} + graphiteFormat = "__g%d__" + + // Number of pre-formatted key names to generate in the init() function. + numPreFormattedTagNames = 128 +) + +var ( + // Should never be modified after init(). + preFormattedTagNames [][]byte +) + +func init() { + for i := 0; i < numPreFormattedTagNames; i++ { + name := generateTagName(i) + preFormattedTagNames = append(preFormattedTagNames, name) + } +} + +// TagName gets a preallocated or generate a tag name for the given graphite +// path index. +func TagName(idx int) []byte { + if idx < len(preFormattedTagNames) { + return preFormattedTagNames[idx] + } + + return []byte(fmt.Sprintf(graphiteFormat, idx)) +} + +func generateTagName(idx int) []byte { + return []byte(fmt.Sprintf(graphiteFormat, idx)) +} diff --git a/src/query/graphite/graphite/tags_test.go b/src/query/graphite/graphite/tags_test.go new file mode 100644 index 0000000000..c09cac246b --- /dev/null +++ b/src/query/graphite/graphite/tags_test.go @@ -0,0 +1,35 @@ +// Copyright (c) 2019 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 graphite + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestTagName(t *testing.T) { + for i := 0; i < 2*numPreFormattedTagNames; i++ { + expected := []byte("__g" + fmt.Sprint(i) + "__") + require.Equal(t, expected, TagName(i)) + } +} diff --git a/src/query/graphite/graphite/testdata/no-results.json b/src/query/graphite/graphite/testdata/no-results.json new file mode 100644 index 0000000000..174bbdc859 --- /dev/null +++ b/src/query/graphite/graphite/testdata/no-results.json @@ -0,0 +1,367 @@ +[ + { + "target": "the.quick.brown.fox.jumped.over-the.lazy_dog.once.again.p50", + "datapoints": [ + [null, 1430835420], + [null, 1430835430], + [null, 1430835440], + [null, 1430835450], + [null, 1430835460], + [null, 1430835470], + [null, 1430835480], + [null, 1430835490], + [null, 1430835500], + [null, 1430835510], + [null, 1430835520], + [null, 1430835530], + [null, 1430835540], + [null, 1430835550], + [null, 1430835560], + [null, 1430835570], + [null, 1430835580], + [null, 1430835590], + [null, 1430835600], + [null, 1430835610], + [null, 1430835620], + [null, 1430835630], + [null, 1430835640], + [null, 1430835650], + [null, 1430835660], + [null, 1430835670], + [null, 1430835680], + [null, 1430835690], + [null, 1430835700], + [null, 1430835710], + [null, 1430835720], + [null, 1430835730], + [null, 1430835740], + [null, 1430835750], + [null, 1430835760], + [null, 1430835770], + [null, 1430835780], + [null, 1430835790], + [null, 1430835800], + [null, 1430835810], + [null, 1430835820], + [null, 1430835830], + [null, 1430835840], + [null, 1430835850], + [null, 1430835860], + [null, 1430835870], + [null, 1430835880], + [null, 1430835890], + [null, 1430835900], + [null, 1430835910], + [null, 1430835920], + [null, 1430835930], + [null, 1430835940], + [null, 1430835950], + [null, 1430835960], + [null, 1430835970], + [null, 1430835980], + [null, 1430835990], + [null, 1430836000], + [null, 1430836010], + [null, 1430836020], + [null, 1430836030], + [null, 1430836040], + [null, 1430836050], + [null, 1430836060], + [null, 1430836070], + [null, 1430836080], + [null, 1430836090], + [null, 1430836100], + [null, 1430836110], + [null, 1430836120], + [null, 1430836130], + [null, 1430836140], + [null, 1430836150], + [null, 1430836160], + [null, 1430836170], + [null, 1430836180], + [null, 1430836190], + [null, 1430836200], + [null, 1430836210], + [null, 1430836220], + [null, 1430836230], + [null, 1430836240], + [null, 1430836250], + [null, 1430836260], + [null, 1430836270], + [null, 1430836280], + [null, 1430836290], + [null, 1430836300], + [null, 1430836310], + [null, 1430836320], + [null, 1430836330], + [null, 1430836340], + [null, 1430836350], + [null, 1430836360], + [null, 1430836370], + [null, 1430836380], + [null, 1430836390], + [null, 1430836400], + [null, 1430836410], + [null, 1430836420], + [null, 1430836430], + [null, 1430836440], + [null, 1430836450], + [null, 1430836460], + [null, 1430836470], + [null, 1430836480], + [null, 1430836490], + [null, 1430836500], + [null, 1430836510], + [null, 1430836520], + [null, 1430836530], + [null, 1430836540], + [null, 1430836550], + [null, 1430836560], + [null, 1430836570], + [null, 1430836580], + [null, 1430836590], + [null, 1430836600], + [null, 1430836610], + [null, 1430836620], + [null, 1430836630], + [null, 1430836640], + [null, 1430836650], + [null, 1430836660], + [null, 1430836670], + [null, 1430836680], + [null, 1430836690], + [null, 1430836700], + [null, 1430836710], + [null, 1430836720], + [null, 1430836730], + [null, 1430836740], + [null, 1430836750], + [null, 1430836760], + [null, 1430836770], + [null, 1430836780], + [null, 1430836790], + [null, 1430836800], + [null, 1430836810], + [null, 1430836820], + [null, 1430836830], + [null, 1430836840], + [null, 1430836850], + [null, 1430836860], + [null, 1430836870], + [null, 1430836880], + [null, 1430836890], + [null, 1430836900], + [null, 1430836910], + [null, 1430836920], + [null, 1430836930], + [null, 1430836940], + [null, 1430836950], + [null, 1430836960], + [null, 1430836970], + [null, 1430836980], + [null, 1430836990], + [null, 1430837000], + [null, 1430837010], + [null, 1430837020], + [null, 1430837030], + [null, 1430837040], + [null, 1430837050], + [null, 1430837060], + [null, 1430837070], + [null, 1430837080], + [null, 1430837090], + [null, 1430837100], + [null, 1430837110], + [null, 1430837120], + [null, 1430837130], + [null, 1430837140], + [null, 1430837150], + [null, 1430837160], + [null, 1430837170], + [null, 1430837180], + [null, 1430837190], + [null, 1430837200], + [null, 1430837210], + [null, 1430837220], + [null, 1430837230], + [null, 1430837240], + [null, 1430837250], + [null, 1430837260], + [null, 1430837270], + [null, 1430837280], + [null, 1430837290], + [null, 1430837300], + [null, 1430837310], + [null, 1430837320], + [null, 1430837330], + [null, 1430837340], + [null, 1430837350], + [null, 1430837360], + [null, 1430837370], + [null, 1430837380], + [null, 1430837390], + [null, 1430837400], + [null, 1430837410], + [null, 1430837420], + [null, 1430837430], + [null, 1430837440], + [null, 1430837450], + [null, 1430837460], + [null, 1430837470], + [null, 1430837480], + [null, 1430837490], + [null, 1430837500], + [null, 1430837510], + [null, 1430837520], + [null, 1430837530], + [null, 1430837540], + [null, 1430837550], + [null, 1430837560], + [null, 1430837570], + [null, 1430837580], + [null, 1430837590], + [null, 1430837600], + [null, 1430837610], + [null, 1430837620], + [null, 1430837630], + [null, 1430837640], + [null, 1430837650], + [null, 1430837660], + [null, 1430837670], + [null, 1430837680], + [null, 1430837690], + [null, 1430837700], + [null, 1430837710], + [null, 1430837720], + [null, 1430837730], + [null, 1430837740], + [null, 1430837750], + [null, 1430837760], + [null, 1430837770], + [null, 1430837780], + [null, 1430837790], + [null, 1430837800], + [null, 1430837810], + [null, 1430837820], + [null, 1430837830], + [null, 1430837840], + [null, 1430837850], + [null, 1430837860], + [null, 1430837870], + [null, 1430837880], + [null, 1430837890], + [null, 1430837900], + [null, 1430837910], + [null, 1430837920], + [null, 1430837930], + [null, 1430837940], + [null, 1430837950], + [null, 1430837960], + [null, 1430837970], + [null, 1430837980], + [null, 1430837990], + [null, 1430838000], + [null, 1430838010], + [null, 1430838020], + [null, 1430838030], + [null, 1430838040], + [null, 1430838050], + [null, 1430838060], + [null, 1430838070], + [null, 1430838080], + [null, 1430838090], + [null, 1430838100], + [null, 1430838110], + [null, 1430838120], + [null, 1430838130], + [null, 1430838140], + [null, 1430838150], + [null, 1430838160], + [null, 1430838170], + [null, 1430838180], + [null, 1430838190], + [null, 1430838200], + [null, 1430838210], + [null, 1430838220], + [null, 1430838230], + [null, 1430838240], + [null, 1430838250], + [null, 1430838260], + [null, 1430838270], + [null, 1430838280], + [null, 1430838290], + [null, 1430838300], + [null, 1430838310], + [null, 1430838320], + [null, 1430838330], + [null, 1430838340], + [null, 1430838350], + [null, 1430838360], + [null, 1430838370], + [null, 1430838380], + [null, 1430838390], + [null, 1430838400], + [null, 1430838410], + [null, 1430838420], + [null, 1430838430], + [null, 1430838440], + [null, 1430838450], + [null, 1430838460], + [null, 1430838470], + [null, 1430838480], + [null, 1430838490], + [null, 1430838500], + [null, 1430838510], + [null, 1430838520], + [null, 1430838530], + [null, 1430838540], + [null, 1430838550], + [null, 1430838560], + [null, 1430838570], + [null, 1430838580], + [null, 1430838590], + [null, 1430838600], + [null, 1430838610], + [null, 1430838620], + [null, 1430838630], + [null, 1430838640], + [null, 1430838650], + [null, 1430838660], + [null, 1430838670], + [null, 1430838680], + [null, 1430838690], + [null, 1430838700], + [null, 1430838710], + [null, 1430838720], + [null, 1430838730], + [null, 1430838740], + [null, 1430838750], + [null, 1430838760], + [null, 1430838770], + [null, 1430838780], + [null, 1430838790], + [null, 1430838800], + [null, 1430838810], + [null, 1430838820], + [null, 1430838830], + [null, 1430838840], + [null, 1430838850], + [null, 1430838860], + [null, 1430838870], + [null, 1430838880], + [null, 1430838890], + [null, 1430838900], + [null, 1430838910], + [null, 1430838920], + [null, 1430838930], + [null, 1430838940], + [null, 1430838950], + [null, 1430838960], + [null, 1430838970], + [null, 1430838980], + [null, 1430838990], + [null, 1430839000], + [null, 1430839010] + ] + } +] diff --git a/src/query/graphite/graphite/timespec.go b/src/query/graphite/graphite/timespec.go new file mode 100644 index 0000000000..4d64b44bd6 --- /dev/null +++ b/src/query/graphite/graphite/timespec.go @@ -0,0 +1,127 @@ +// Copyright (c) 2019 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 graphite + +import ( + "fmt" + "regexp" + "strconv" + "strings" + "time" + + "github.com/m3db/m3/src/query/graphite/errors" +) + +var reRelativeTime = regexp.MustCompile(`(?i)^\-([0-9]+)(s|min|h|d|w|mon|y)$`) + +var periods = map[string]time.Duration{ + "s": time.Second, + "min": time.Minute, + "h": time.Hour, + "d": time.Hour * 24, + "w": time.Hour * 24 * 7, + "mon": time.Hour * 24 * 30, + "y": time.Hour * 24 * 365, +} + +// on Jan 2 15:04:05 -0700 MST 2006 +var formats = []string{ + "15:04_060102", + "15:04_20060102", + "15:04_01/02/06", + "15:04_02.01.06", + "02.01.06", + "01/02/06", + "060102", + "20060102", +} + +// use init to rewrite formats to bypass bug in time.Parse +func init() { + for i := range formats { + formats[i] = bypassTimeParseBug(formats[i]) + } +} + +func bypassTimeParseBug(s string) string { + // NB(jayp): Looks like there is a bug in Golang's time.Parse when handing format strings + // with _2 in the format string. Here is a snippet that exhibits this issue: + // t, e := time.Parse("15:04_20060102", "14:38_20150618") + // We replace underscores with space to bypass this bug. + return strings.Replace(s, "_", " ", -1) +} + +// FormatTime translates a time.Time until a Graphite from/until string +func FormatTime(t time.Time) string { + return t.Format(formats[0]) +} + +// ParseTime translates a Graphite from/until string into a timestamp relative to the provide time +func ParseTime(s string, now time.Time, absoluteOffset time.Duration) (time.Time, error) { + if len(s) == 0 { + return now, errors.NewInvalidParamsError(fmt.Errorf("time cannot be empty")) + } + + if s == "now" { + return now, nil + } + + if m := reRelativeTime.FindStringSubmatch(s); len(m) != 0 { + timePast, err := strconv.ParseInt(m[1], 10, 32) + if err != nil { + return now, errors.NewInvalidParamsError(fmt.Errorf("invalid relative time %v", err)) + } + + period := periods[strings.ToLower(m[2])] + return now.Add(-1 * time.Duration(timePast) * period), nil + } + + newS := bypassTimeParseBug(s) + for _, format := range formats { + t, err := time.Parse(format, newS) + if err == nil { + // Absolute time passed in, applying offset + return t.Add(absoluteOffset), nil + } + } + + n, err := strconv.ParseInt(s, 10, 64) + if err == nil { + return time.Unix(n, 0).UTC(), nil + } + + return now, err +} + +// ParseDuration parses a duration +func ParseDuration(s string) (time.Duration, error) { + if m := reRelativeTime.FindStringSubmatch(s); len(m) != 0 { + timePast, err := strconv.ParseInt(m[1], 10, 32) + if err != nil { + return 0, errors.NewInvalidParamsError(fmt.Errorf("invalid relative time %v", err)) + } + + period := periods[strings.ToLower(m[2])] + return -1 * time.Duration(timePast) * period, nil + } + + return 0, errors.NewInvalidParamsError(fmt.Errorf("invalid relative time %s", s)) +} diff --git a/src/query/graphite/graphite/timespec_test.go b/src/query/graphite/graphite/timespec_test.go new file mode 100644 index 0000000000..3f6da56f7a --- /dev/null +++ b/src/query/graphite/graphite/timespec_test.go @@ -0,0 +1,106 @@ +// Copyright (c) 2019 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 graphite + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +var relativeTo = time.Date(2013, time.April, 3, 4, 5, 0, 0, time.UTC) + +func TestParseTime(t *testing.T) { + + tests := []struct { + timespec string + expectedTime time.Time + }{ + {"-4h", relativeTo.Add(-1 * 4 * time.Hour)}, + {"-35MIN", relativeTo.Add(-1 * 35 * time.Minute)}, + {"06:12_07.03.14", time.Date(2014, time.March, 7, 6, 12, 0, 0, time.UTC)}, + {"06:12_03/07/14", time.Date(2014, time.March, 7, 6, 12, 0, 0, time.UTC)}, + {"06:12_140307", time.Date(2014, time.March, 7, 6, 12, 0, 0, time.UTC)}, + {"14:38_20150618", time.Date(2015, time.June, 18, 14, 38, 0, 0, time.UTC)}, + {"07.03.14", time.Date(2014, time.March, 7, 0, 0, 0, 0, time.UTC)}, + {"03/07/14", time.Date(2014, time.March, 7, 0, 0, 0, 0, time.UTC)}, + {"20140307", time.Date(2014, time.March, 7, 0, 0, 0, 0, time.UTC)}, + {"140307", time.Date(2014, time.March, 7, 0, 0, 0, 0, time.UTC)}, + {"1432581620", time.Date(2015, time.May, 25, 19, 20, 20, 0, time.UTC)}, + } + + for _, test := range tests { + s := test.timespec + parsed, err := ParseTime(s, relativeTo, 0) + assert.Nil(t, err, "error parsing %s", s) + assert.Equal(t, test.expectedTime, parsed, "incorrect parsed value for %s", s) + } +} + +func TestParseDuration(t *testing.T) { + tests := []struct { + timespec string + expectedDuration time.Duration + }{ + {"-4h", -4 * time.Hour}, + {"-35MIN", -35 * time.Minute}, + {"-10s", -10 * time.Second}, + } + + for _, test := range tests { + s := test.timespec + parsed, err := ParseDuration(s) + assert.Nil(t, err, "error parsing %s", s) + assert.Equal(t, test.expectedDuration, parsed, "incorrect parsed value for %s", s) + } +} + +func TestParseDurationErrors(t *testing.T) { + tests := []string{ + "10s", + "-10.5h", + } + + for _, test := range tests { + parsed, err := ParseDuration(test) + assert.Error(t, err) + assert.Equal(t, time.Duration(0), parsed) + } +} + +func TestAbsoluteOffset(t *testing.T) { + tests := []struct { + timespec string + expectedTime time.Time + }{ + {"-35MIN", relativeTo.Add(-1 * 35 * time.Minute)}, + {"14:12_07.03.14", time.Date(2014, time.March, 7, 7, 12, 0, 0, time.UTC)}, + {"03/07/14", time.Date(2014, time.March, 6, 17, 0, 0, 0, time.UTC)}, + } + + for _, test := range tests { + s := test.timespec + parsed, err := ParseTime(s, relativeTo, -7*time.Hour) + assert.Nil(t, err, "error parsing %s", s) + assert.Equal(t, test.expectedTime, parsed, "incorrect parsed value for %s", s) + } +} diff --git a/src/query/graphite/lexer/lexer.go b/src/query/graphite/lexer/lexer.go new file mode 100644 index 0000000000..2fcdbe37cb --- /dev/null +++ b/src/query/graphite/lexer/lexer.go @@ -0,0 +1,483 @@ +// Copyright (c) 2019 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 lexer + +import ( + "fmt" + "strings" + "unicode/utf8" + + "github.com/m3db/m3/src/query/graphite/graphite" +) + +// TokenType defines the type of identifier recognized by the Lexer. +type TokenType int + +const ( + // Error is what you get when the lexer fails to grok the input. + Error TokenType = iota + // Identifier is a symbol confining to C-style variable naming rules. + Identifier + // Pattern is a regex-ish pattern, accepts the following special chars: [{.*}]. + Pattern + // Number is a numeral, including floats. + Number + // String is set of characters wrapped by double quotes. + String + // LParenthesis is the left parenthesis "(". + LParenthesis + // RParenthesis is the right parenthesis ")". + RParenthesis + // Colon is the ":" symbol. + Colon + // NotOperator is the exclamation sign - "!" symbol. + NotOperator + // Comma is a punctuation mark. + Comma + // Equal is the "=" symbol. + Equal + + // True is Boolean true. + True + // False is Boolean false. + False +) + +func (tt TokenType) String() string { + switch tt { + case Error: + return "Error" + case Identifier: + return "Identifier" + case Pattern: + return "Pattern" + case Number: + return "Number" + case String: + return "String" + case LParenthesis: + return "LParenthesis" + case RParenthesis: + return "RParenthesis" + case Colon: + return "Colon" + case NotOperator: + return "NotOperator" + case Comma: + return "Comma" + case Equal: + return "Equal" + case True: + return "True" + case False: + return "False" + } + return fmt.Sprintf("UnknownToken(%d)", int(tt)) +} + +var symbols = map[rune]TokenType{ + '(': LParenthesis, + ')': RParenthesis, + ':': Colon, + '!': NotOperator, + ',': Comma, + '=': Equal, +} + +// Token is a token, doh! +type Token struct { + tokenType TokenType + value string +} + +// TokenType returns the type of token consumed. +func (t Token) TokenType() TokenType { + return t.tokenType +} + +// Value returns the string representation of the token as needed. +func (t Token) Value() string { + return t.value +} + +const ( + uppercaseLetters = "ABCDEFGHIJKLMNOPQRSTUVWXYZ" + lowercaseLetters = "abcdefghijklmnopqrstuvwxyz" + digits = "0123456789" + exponentRunes = "eE" + identifierStartRunes = uppercaseLetters + lowercaseLetters + "_" + "$" + identifierRunes = identifierStartRunes + digits + "-" + signs = "+-" +) + +// Lexer breaks an input stream into a group of lexical elements. +type Lexer struct { + tokens chan *Token + s string + start int + pos int + width int + reservedIdentifiers map[string]TokenType +} + +const ( + eof rune = 0 +) + +// NewLexer returns a lexer and an output channel for tokens. +func NewLexer(s string, reservedIdentifiers map[string]TokenType) (*Lexer, chan *Token) { + tokens := make(chan *Token) + return &Lexer{s: s, tokens: tokens, reservedIdentifiers: reservedIdentifiers}, tokens +} + +// Run consumes the input to produce a token stream. +func (l *Lexer) Run() { + for l.lex() { + } + close(l.tokens) +} + +func (l *Lexer) lex() bool { + l.skipWhitespace() + + r := l.next() + if r == eof { + return false + } + + if r == '"' || r == '\'' { + return l.quotedString(r) + } + + if r == '+' || r == '-' { + return l.positiveOrNegativeNumber() + } + + if r == '.' { + return l.fractionalOnlyNumber() + } + + if strings.ContainsRune(digits, r) { + return l.numberOrPattern() + } + + if strings.ContainsRune(identifierStartRunes, r) { + return l.identifierOrPattern() + } + + if strings.ContainsRune("{[*.", r) { + l.backup() + return l.pattern() + } + + sym, ok := symbols[r] + if !ok { + return l.errorf("unexpected character %c", r) + } + + l.emit(sym) + return true +} + +func (l *Lexer) eof() bool { + l.skipWhitespace() + return l.pos >= len(l.s) +} + +func (l *Lexer) positiveOrNegativeNumber() bool { + if !l.acceptRun(digits) { + return l.unexpected(digits) + } + + if l.accept(".") { + return l.fractionalPart() + } + + l.emit(Number) + return true +} + +func (l *Lexer) fractionalOnlyNumber() bool { + if !l.acceptRun(digits) { + return l.unexpected(digits) + } + if l.accept(exponentRunes) { + return l.exponentPart() + } + l.emit(Number) + return true +} + +func (l *Lexer) fractionalPart() bool { + l.acceptRun(digits) + l.emit(Number) + return true +} + +func (l *Lexer) exponentPart() bool { + l.accept(signs) + if !l.acceptRun(digits) { + return l.unexpected(digits) + } + l.emit(Number) + return true +} + +func (l *Lexer) numberOrPattern() bool { + l.acceptRun(digits) + if l.accept(".") { + return l.fractionalPartOrPattern() + } + + r := l.next() + if r != eof { + l.backup() + } + if l.accept(exponentRunes) { + return l.exponentPart() + } + if strings.ContainsRune("{[*-"+identifierStartRunes, r) { + return l.pattern() + } + + l.emit(Number) + return true +} + +func (l *Lexer) fractionalPartOrPattern() bool { + l.acceptRun(digits) + + r := l.next() + if r != eof { + l.backup() + } + if l.accept(exponentRunes) { + return l.exponentPart() + } + if strings.ContainsRune("{[*-."+identifierStartRunes, r) { + return l.pattern() + } + + l.emit(Number) + return true +} + +func (l *Lexer) identifierOrPattern() bool { + l.acceptRun(identifierRunes) + + r := l.next() + if r != eof { + l.backup() + } + if strings.ContainsRune("{[*.-", r) { + return l.pattern() + } + + // Check if idenitifer is one of the reserved identifiers. + for text, identifier := range l.reservedIdentifiers { + if strings.ToLower(l.currentVal()) == text { + l.emit(identifier) + return true + } + } + + l.emit(Identifier) + return true +} + +// NB(jayp): initialized by init(). +var groupingEndsToStarts = map[rune]rune{} + +var groupingStartsToEnds = map[rune]rune{ + '{': '}', + '[': ']', +} + +func (l *Lexer) pattern() bool { + // rune(0) indicates pattern is not in a group. + groupStartStack := []rune{rune(0)} + for { + r := l.next() + + // Start of a group. + if _, ok := groupingStartsToEnds[r]; ok { + // Start another group. + groupStartStack = append(groupStartStack, r) + continue + } + + // End of a group. + if groupStart, ok := groupingEndsToStarts[r]; ok { + // Unwind group. + if groupStart != groupStartStack[len(groupStartStack)-1] { + return l.errorf("encountered unbalanced end of group %c in pattern %s", + r, l.currentVal()) + } + groupStartStack = groupStartStack[:len(groupStartStack)-1] + continue + } + + if strings.ContainsRune(graphite.ValidIdentifierRunes+".?*", r) { + continue + } + + // Commas are part of the pattern if they appear in a group + if r == ',' && groupStartStack[len(groupStartStack)-1] != 0 { + continue + } + + // Everything else is the end of the pattern. + if groupStartStack[len(groupStartStack)-1] != 0 { + return l.errorf("end of pattern %s reached while still in group %c", + l.currentVal(), groupStartStack[len(groupStartStack)-1]) + } + + if r != eof { + l.backup() + } + l.emit(Pattern) + return true + } +} + +func (l *Lexer) quotedString(quoteMark rune) bool { + var s []rune + escaped := false + for { + r := l.next() + if r == eof { + return l.errorf("reached end of input while processing string %s", l.currentVal()) + } + + if !escaped && r == quoteMark { + l.emitToken(String, string(s)) + l.consumeVal() + return true + } + + if !escaped && r == '\\' { + // TODO: Want to omit this from the output. + escaped = true + continue + } + + if escaped && strings.ContainsRune(digits, r) { + // if backslash is followed by a digit, we add the backslash back + s = append(s, '\\') + } + + s = append(s, r) + escaped = false + } +} + +func (l *Lexer) unexpected(expected string) bool { + r := l.next() + l.backup() + return l.errorf("expected one of %s, found %c", expected, r) +} + +func (l *Lexer) skipWhitespace() { + l.acceptRun(" \t\r\n") + l.ignore() +} + +func (l *Lexer) next() (r rune) { + if l.pos >= len(l.s) { + l.width = 0 + return eof + } + + r, l.width = utf8.DecodeRuneInString(l.s[l.pos:]) + l.pos += l.width + return r +} + +func (l *Lexer) ignore() { + l.start = l.pos +} + +func (l *Lexer) backup() { + l.pos-- +} + +func (l *Lexer) accept(valid string) bool { + r := l.next() + if r != eof && strings.ContainsRune(valid, r) { + return true + } + + if r != eof { + l.backup() + } + return false +} + +func (l *Lexer) acceptRun(valid string) bool { + matched := false + + r := l.next() + for strings.ContainsRune(valid, r) && r != eof { + matched = true + r = l.next() + } + + if r != eof { + l.backup() + } + + return matched +} + +func (l *Lexer) currentVal() string { + return l.s[l.start:l.pos] +} + +func (l *Lexer) consumeVal() string { + s := l.currentVal() + l.start = l.pos + return s +} + +func (l *Lexer) emit(tt TokenType) { + l.emitToken(tt, l.consumeVal()) +} + +func (l *Lexer) emitToken(tt TokenType, val string) { + l.tokens <- &Token{ + tokenType: tt, + value: val, + } +} + +func (l *Lexer) errorf(msg string, args ...interface{}) bool { + l.tokens <- &Token{ + tokenType: Error, + value: fmt.Sprintf(msg, args...), + } + return false +} + +func init() { + for start, end := range groupingStartsToEnds { + groupingEndsToStarts[end] = start + } +} diff --git a/src/query/graphite/lexer/lexer_test.go b/src/query/graphite/lexer/lexer_test.go new file mode 100644 index 0000000000..41138e70d1 --- /dev/null +++ b/src/query/graphite/lexer/lexer_test.go @@ -0,0 +1,162 @@ +// Copyright (c) 2019 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 lexer + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +type lexerTestData struct { + input string + expectedTokens []Token + reservedIdents map[string]TokenType +} + +func TestLexer(t *testing.T) { + lexerTestInput := []lexerTestData{ + {"call09", []Token{{Identifier, "call09"}}, nil}, + {"sortByName(foo.bar.zed)", []Token{ + {Identifier, "sortByName"}, + {LParenthesis, "("}, + {Pattern, "foo.bar.zed"}, + {RParenthesis, ")"}}, nil}, + {"foo.'bar<1001>'.baz", + []Token{{Pattern, "foo.'bar<1001>'.baz"}}, nil}, + {"a.{b,c,d}.node[0-2].qux.*", + []Token{{Pattern, "a.{b,c,d}.node[0-2].qux.*"}}, nil}, + {"qaz{0[3-9],1[0-9],20}", + []Token{{Pattern, "qaz{0[3-9],1[0-9],20}"}}, nil}, + {"qaz{0[3-9],1[0-9],20}.bar", + []Token{{Pattern, "qaz{0[3-9],1[0-9],20}.bar"}}, nil}, + {"stats.foo.counts.bar.baz.status_code.?XX", + []Token{{Pattern, "stats.foo.counts.bar.baz.status_code.?XX"}}, nil}, + {"456.03", []Token{{Number, "456.03"}}, nil}, + {"foo:bar", []Token{ + {Identifier, "foo"}, + {Colon, ":"}, + {Identifier, "bar"}}, nil}, + {"foo:bar baz : q*x", []Token{ + {Identifier, "foo"}, + {Colon, ":"}, + {Identifier, "bar"}, + {Identifier, "baz"}, + {Colon, ":"}, + {Pattern, "q*x"}}, nil}, + {"!service:dispatch.*", []Token{ + {NotOperator, "!"}, + {Identifier, "service"}, + {Colon, ":"}, + {Pattern, "dispatch.*"}}, nil}, + {"\"Whatever man\"", []Token{{String, "Whatever man"}}, nil}, // double quoted string + // no need to escape single quotes within double quoted string + {"\"Whatever 'man'\"", []Token{{String, "Whatever 'man'"}}, nil}, + // escape double quotes within double quoted string + {"\"Whatever \\\"man\\\"\"", []Token{{String, "Whatever \"man\""}}, nil}, + {"'Whatever man'", []Token{{String, "Whatever man"}}, nil}, // single quoted string + // no need to escape double quote within single quoted strings (start boundary), but may + // do it if so desired (end boundary) + {"'Whatever \"man\\\"'", []Token{{String, "Whatever \"man\""}}, nil}, + {" call09(a.{b,c,d}.node[0-2].qux.*, a{e,g}, 4546.abc, 45ahj, " + + "\"Hello there \\\"Good \\\\ Sir\\\" \", +20, 39540.459,-349845,.393) ", + []Token{ + {Identifier, "call09"}, + {LParenthesis, "("}, + {Pattern, "a.{b,c,d}.node[0-2].qux.*"}, + {Comma, ","}, + {Pattern, "a{e,g}"}, + {Comma, ","}, + {Pattern, "4546.abc"}, + {Comma, ","}, + {Pattern, "45ahj"}, + {Comma, ","}, + {String, "Hello there \"Good \\ Sir\" "}, + {Comma, ","}, + {Number, "+20"}, + {Comma, ","}, + {Number, "39540.459"}, + {Comma, ","}, + {Number, "-349845"}, + {Comma, ","}, + {Number, ".393"}, + {RParenthesis, ")"}}, nil}, + {`aliasSub(stats.foo.timers.scream.scream.views.quz.end_to_end_latency.p95, ` + + `'stats.(.*).timers.scream.scream.views.quz.(.*)', '\1.\2')`, + []Token{ + {Identifier, "aliasSub"}, + {LParenthesis, "("}, + {Pattern, "stats.foo.timers.scream.scream.views.quz.end_to_end_latency.p95"}, + {Comma, ","}, + {String, "stats.(.*).timers.scream.scream.views.quz.(.*)"}, + {Comma, ","}, + {String, `\1.\2`}, + {RParenthesis, ")"}}, nil}, + } + + for _, test := range lexerTestInput { + lex, tokens := NewLexer(test.input, test.reservedIdents) + go lex.Run() + + i := 0 + for token := range tokens { + require.True(t, i < len(test.expectedTokens), + "received more tokens than expected for %s", test.input) + assert.Equal(t, &test.expectedTokens[i], token, "incorrect token %d for %s", + i, test.input) + i++ + } + + assert.True(t, lex.eof(), "did not reach eof for %s", test.input) + } +} + +func TestLexerErrors(t *testing.T) { + badLines := [][]string{ + {"\"this is \\\"unterminated", + "reached end of input while processing string \"this is \\\"unterminated"}, + {"ajs.djd]", // group closed without open + "encountered unbalanced end of group ] in pattern ajs.djd]"}, + {"{dfklf]", // mismatch between open and close of group + "encountered unbalanced end of group ] in pattern {dfklf]"}, + {"{[ajs.djd}]", // flipped close groups + "encountered unbalanced end of group } in pattern {[ajs.djd}"}, + {"{unclosed", "end of pattern {unclosed reached while still in group {"}, // unclosed group + {"+a", "expected one of 0123456789, found a"}, // plus with no number + {"-b", "expected one of 0123456789, found b"}, // minus with no number + {".c", "expected one of 0123456789, found c"}, // digit with no number + {"^", "unexpected character ^"}, // unknown symbol + } + + for _, badLine := range badLines { + l, tokens := NewLexer(badLine[0], nil) + go l.Run() + + expected := &Token{Error, badLine[1]} + actual := <-tokens + assert.Equal(t, expected, actual, "%s did not result in error", badLine[0]) + + // Should have closed the channel after the error + actual = <-tokens + assert.Nil(t, actual) + } +} diff --git a/src/query/graphite/native/aggregation_functions.go b/src/query/graphite/native/aggregation_functions.go new file mode 100644 index 0000000000..fbbcf565f0 --- /dev/null +++ b/src/query/graphite/native/aggregation_functions.go @@ -0,0 +1,416 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "math" + "strings" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +func wrapPathExpr(wrapper string, series ts.SeriesList) string { + return fmt.Sprintf("%s(%s)", wrapper, joinPathExpr(series)) +} + +// sumSeries adds metrics together and returns the sum at each datapoint. +// If the time series have different intervals, the coarsest interval will be used. +func sumSeries(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + return combineSeries(ctx, series, wrapPathExpr("sumSeries", ts.SeriesList(series)), ts.Sum) +} + +// diffSeries subtracts all but the first series from the first series. +// If the time series have different intervals, the coarsest interval will be used. +func diffSeries(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + transformedSeries := series + numSeries := len(series.Values) + if numSeries > 1 { + transformedSeries.Values = make([]*ts.Series, numSeries) + transformedSeries.Values[0] = series.Values[0] + for i := 1; i < len(series.Values); i++ { + res, err := transform( + ctx, + singlePathSpec{Values: []*ts.Series{series.Values[i]}}, + func(n string) string { return n }, + common.MaintainNaNTransformer(func(v float64) float64 { return -v }), + ) + if err != nil { + return ts.SeriesList{}, err + } + transformedSeries.Values[i] = res.Values[0] + } + } + + return combineSeries(ctx, transformedSeries, wrapPathExpr("diffSeries", ts.SeriesList(series)), ts.Sum) +} + +// multiplySeries multiplies metrics together and returns the product at each datapoint. +// If the time series have different intervals, the coarsest interval will be used. +func multiplySeries(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + return combineSeries(ctx, series, wrapPathExpr("multiplySeries", ts.SeriesList(series)), ts.Mul) +} + +// averageSeries takes a list of series and returns a new series containing the +// average of all values at each datapoint. +func averageSeries(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + return combineSeries(ctx, series, wrapPathExpr("averageSeries", ts.SeriesList(series)), ts.Avg) +} + +// minSeries takes a list of series and returns a new series containing the +// minimum value across the series at each datapoint +func minSeries(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + return combineSeries(ctx, series, wrapPathExpr("minSeries", ts.SeriesList(series)), ts.Min) +} + +// maxSeries takes a list of series and returns a new series containing the +// maximum value across the series at each datapoint +func maxSeries(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + return combineSeries(ctx, series, wrapPathExpr("maxSeries", ts.SeriesList(series)), ts.Max) +} + +// divideSeries divides one series list by another series +func divideSeries(ctx *common.Context, dividendSeriesList, divisorSeriesList singlePathSpec) (ts.SeriesList, error) { + if len(divisorSeriesList.Values) != 1 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "divideSeries second argument must reference exactly one series but instead has %d", + len(divisorSeriesList.Values))) + return ts.SeriesList{}, err + } + if len(dividendSeriesList.Values) == 0 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "divideSeries first argument must reference at least one series")) + return ts.SeriesList{}, err + } + + divisorSeries := divisorSeriesList.Values[0] + results := make([]*ts.Series, len(dividendSeriesList.Values)) + for idx, dividendSeries := range dividendSeriesList.Values { + normalized, minBegin, _, lcmMillisPerStep, err := common.Normalize(ctx, ts.SeriesList{ + Values: []*ts.Series{dividendSeries, divisorSeries}, + }) + if err != nil { + return ts.SeriesList{}, err + } + // NB(bl): Normalized must give back exactly two series of the same length. + dividend, divisor := normalized.Values[0], normalized.Values[1] + numSteps := dividend.Len() + vals := ts.NewValues(ctx, lcmMillisPerStep, numSteps) + for i := 0; i < numSteps; i++ { + dividendVal := dividend.ValueAt(i) + divisorVal := divisor.ValueAt(i) + if !math.IsNaN(dividendVal) && !math.IsNaN(divisorVal) && divisorVal != 0 { + value := dividendVal / divisorVal + vals.SetValueAt(i, value) + } + } + name := fmt.Sprintf("divideSeries(%s,%s)", dividend.Name(), divisor.Name()) + quotientSeries := ts.NewSeries(ctx, name, minBegin, vals) + results[idx] = quotientSeries + } + + r := ts.SeriesList(dividendSeriesList) + r.Values = results + return r, nil +} + +// averageSeriesWithWildcards splits the given set of series into sub-groupings +// based on wildcard matches in the hierarchy, then averages the values in each +// grouping +func averageSeriesWithWildcards( + ctx *common.Context, + series singlePathSpec, + positions ...int, +) (ts.SeriesList, error) { + return combineSeriesWithWildcards(ctx, series, positions, averageSpecificationFunc, ts.Avg) +} + +// sumSeriesWithWildcards splits the given set of series into sub-groupings +// based on wildcard matches in the hierarchy, then sums the values in each +// grouping +func sumSeriesWithWildcards( + ctx *common.Context, + series singlePathSpec, + positions ...int, +) (ts.SeriesList, error) { + return combineSeriesWithWildcards(ctx, series, positions, sumSpecificationFunc, ts.Sum) +} + +// combineSeriesWithWildcards splits the given set of series into sub-groupings +// based on wildcard matches in the hierarchy, then combines the values in each +// sub-grouping according to the provided consolidation function +func combineSeriesWithWildcards( + ctx *common.Context, + series singlePathSpec, + positions []int, + sf specificationFunc, + f ts.ConsolidationFunc, +) (ts.SeriesList, error) { + if len(series.Values) == 0 { + return ts.SeriesList(series), nil + } + + var ( + toCombine = make(map[string][]*ts.Series) + wildcards = make(map[int]struct{}) + ) + + for _, position := range positions { + wildcards[position] = struct{}{} + } + + for _, series := range series.Values { + var ( + parts = strings.Split(series.Name(), ".") + newParts = make([]string, 0, len(parts)) + ) + for i, part := range parts { + if _, wildcard := wildcards[i]; !wildcard { + newParts = append(newParts, part) + } + } + + newName := strings.Join(newParts, ".") + toCombine[newName] = append(toCombine[newName], series) + } + + newSeries := make([]*ts.Series, 0, len(toCombine)) + for name, series := range toCombine { + seriesList := ts.SeriesList{Values: series} + combined, err := combineSeries(ctx, multiplePathSpecs(seriesList), name, f) + if err != nil { + return ts.SeriesList{}, err + } + combined.Values[0].Specification = sf(seriesList) + newSeries = append(newSeries, combined.Values...) + } + + r := ts.SeriesList(series) + + r.Values = newSeries + + // Ranging over hash map to create results destroys + // any sort order on the incoming series list + r.SortApplied = false + + return r, nil +} + +// groupByNode takes a serieslist and maps a callback to subgroups within as defined by a common node +// +// &target=groupByNode(foo.by-function.*.*.cpu.load5,2,"sumSeries") +// +// Would return multiple series which are each the result of applying the "sumSeries" function +// to groups joined on the second node (0 indexed) resulting in a list of targets like +// +// sumSeries(foo.by-function.server1.*.cpu.load5),sumSeries(foo.by-function.server2.*.cpu.load5),... +func groupByNode(ctx *common.Context, series singlePathSpec, node int, fname string) (ts.SeriesList, error) { + metaSeries := make(map[string][]*ts.Series) + for _, s := range series.Values { + parts := strings.Split(s.Name(), ".") + + n := node + if n < 0 { + n = len(parts) + n + } + + if n >= len(parts) || n < 0 { + err := errors.NewInvalidParamsError(fmt.Errorf("could not group %s by node %d; not enough parts", s.Name(), node)) + return ts.SeriesList{}, err + } + + key := parts[n] + metaSeries[key] = append(metaSeries[key], s) + } + + if fname == "" { + fname = "sum" + } + + f, fexists := summarizeFuncs[fname] + if !fexists { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("invalid func %s", fname)) + } + + newSeries := make([]*ts.Series, 0, len(metaSeries)) + for key, series := range metaSeries { + seriesList := ts.SeriesList{Values: series} + output, err := combineSeries(ctx, multiplePathSpecs(seriesList), key, f.consolidationFunc) + if err != nil { + return ts.SeriesList{}, err + } + output.Values[0].Specification = f.specificationFunc(seriesList) + newSeries = append(newSeries, output.Values...) + } + + r := ts.SeriesList(series) + + r.Values = newSeries + + // Ranging over hash map to create results destroys + // any sort order on the incoming series list + r.SortApplied = false + + return r, nil +} + +// combineSeries combines multiple series into a single series using a +// consolidation func. If the series use different time intervals, the +// coarsest time will apply. +func combineSeries(ctx *common.Context, + series multiplePathSpecs, + fname string, + f ts.ConsolidationFunc, +) (ts.SeriesList, error) { + if len(series.Values) == 0 { // no data; no work + return ts.SeriesList(series), nil + } + + normalized, start, end, millisPerStep, err := common.Normalize(ctx, ts.SeriesList(series)) + if err != nil { + err := errors.NewInvalidParamsError(fmt.Errorf("combine series error: %v", err)) + return ts.SeriesList{}, err + } + + consolidation := ts.NewConsolidation(ctx, start, end, millisPerStep, f) + for _, s := range normalized.Values { + consolidation.AddSeries(s, ts.Avg) + } + + result := consolidation.BuildSeries(fname, ts.Finalize) + return ts.SeriesList{Values: []*ts.Series{result}}, nil +} + +// weightedAverage takes a series of values and a series of weights and produces a weighted +// average for all values. The corresponding values should share a node as defined by the +// node parameter, 0-indexed. +func weightedAverage( + ctx *common.Context, + input singlePathSpec, + weights singlePathSpec, + node int, +) (ts.SeriesList, error) { + step := math.MaxInt32 + if len(input.Values) > 0 { + step = input.Values[0].MillisPerStep() + } else if len(weights.Values) > 0 { + step = weights.Values[0].MillisPerStep() + } else { + return ts.SeriesList(input), nil + } + + for _, series := range input.Values { + if step != series.MillisPerStep() { + err := errors.NewInvalidParamsError(fmt.Errorf("different step sizes in input series not supported")) + return ts.SeriesList{}, err + } + } + + for _, series := range weights.Values { + if step != series.MillisPerStep() { + err := errors.NewInvalidParamsError(fmt.Errorf("different step sizes in input series not supported")) + return ts.SeriesList{}, err + } + } + + valuesByKey, err := aliasByNode(ctx, input, node) + if err != nil { + return ts.SeriesList{}, err + } + weightsByKey, err := aliasByNode(ctx, weights, node) + if err != nil { + return ts.SeriesList{}, err + } + + type pairedSeries struct { + values *ts.Series + weights *ts.Series + } + + keys := make(map[string]*pairedSeries, len(valuesByKey.Values)) + + for _, series := range valuesByKey.Values { + keys[series.Name()] = &pairedSeries{values: series} + } + + for _, series := range weightsByKey.Values { + if tuple, ok := keys[series.Name()]; ok { + tuple.weights = series + } + } + + productSeries := make([]*ts.Series, 0, len(keys)) + consideredWeights := make([]*ts.Series, 0, len(keys)) + + for key, pair := range keys { + + if pair.weights == nil { + continue // skip - no associated weight series + } + + vals := ts.NewValues(ctx, pair.values.MillisPerStep(), pair.values.Len()) + for i := 0; i < pair.values.Len(); i++ { + v := pair.values.ValueAt(i) + w := pair.weights.ValueAt(i) + vals.SetValueAt(i, v*w) + } + series := ts.NewSeries(ctx, key, pair.values.StartTime(), vals) + productSeries = append(productSeries, series) + consideredWeights = append(consideredWeights, pair.weights) + } + + top, err := sumSeries(ctx, multiplePathSpecs(ts.SeriesList{ + Values: productSeries, + })) + if err != nil { + return ts.SeriesList{}, err + } + + bottom, err := sumSeries(ctx, multiplePathSpecs(ts.SeriesList{ + Values: consideredWeights, + })) + if err != nil { + return ts.SeriesList{}, err + } + + results, err := divideSeries(ctx, singlePathSpec(top), singlePathSpec(bottom)) + if err != nil { + return ts.SeriesList{}, err + } + + return alias(ctx, singlePathSpec(results), "weightedAverage") +} + +// countSeries draws a horizontal line representing the number of nodes found in the seriesList. +func countSeries(ctx *common.Context, seriesList multiplePathSpecs) (ts.SeriesList, error) { + count, err := common.Count(ctx, ts.SeriesList(seriesList), func(series ts.SeriesList) string { + return wrapPathExpr("countSeries", series) + }) + if err != nil { + return ts.SeriesList{}, err + } + + r := ts.SeriesList(seriesList) + r.Values = count.Values + return r, nil +} diff --git a/src/query/graphite/native/aggregation_functions_test.go b/src/query/graphite/native/aggregation_functions_test.go new file mode 100644 index 0000000000..2359a6e269 --- /dev/null +++ b/src/query/graphite/native/aggregation_functions_test.go @@ -0,0 +1,455 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "math" + "sort" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/storage" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var ( + consolidationStartTime = time.Now().Truncate(time.Minute).Add(10 * time.Second) + consolidationEndTime = consolidationStartTime.Add(1 * time.Minute) +) + +func newConsolidationTestSeries() (*common.Context, []*ts.Series) { + ctx := common.NewContext(common.ContextOptions{Start: consolidationStartTime, End: consolidationEndTime}) + + testSeries := []*ts.Series{ + ts.NewSeries(ctx, "a", consolidationStartTime, + ts.NewConstantValues(ctx, 10, 6, 10000)), + ts.NewSeries(ctx, "b", consolidationStartTime.Add(-30*time.Second), + ts.NewConstantValues(ctx, 15, 6, 10000)), + ts.NewSeries(ctx, "c", consolidationStartTime.Add(30*time.Second), + ts.NewConstantValues(ctx, 17, 6, 10000)), + ts.NewSeries(ctx, "d", consolidationStartTime, + ts.NewConstantValues(ctx, 3, 60, 1000)), + } + + return ctx, testSeries +} + +func testAggregatedSeries( + t *testing.T, + f func(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error), + ev1, ev2, ev3, ev4 float64, + errorMessage string, +) { + ctx, consolidationTestSeries := newConsolidationTestSeries() + defer ctx.Close() + + input := ts.SeriesList{Values: consolidationTestSeries} + + r, err := f(ctx, multiplePathSpecs(input)) + require.Nil(t, err) + + series := r.Values + require.Equal(t, 1, len(series)) + + require.Equal(t, consolidationTestSeries[1].StartTime(), series[0].StartTime()) + require.Equal(t, consolidationTestSeries[2].EndTime(), series[0].EndTime()) + require.Equal(t, 12, series[0].Len()) + require.Equal(t, 10000, series[0].MillisPerStep()) + for i := 0; i < 3; i++ { + n := series[0].ValueAt(i) + assert.Equal(t, ev1, n, errorMessage, i) + } + for i := 3; i < 6; i++ { + n := series[0].ValueAt(i) + assert.Equal(t, ev2, n, errorMessage, i) + } + for i := 6; i < 9; i++ { + n := series[0].ValueAt(i) + assert.Equal(t, ev3, n, errorMessage, i) + } + for i := 9; i < 12; i++ { + n := series[0].ValueAt(i) + assert.Equal(t, ev4, n, errorMessage, i) + } + + // nil input -> nil output + for _, in := range [][]*ts.Series{nil, []*ts.Series{}} { + series, err := f(ctx, multiplePathSpecs(ts.SeriesList{ + Values: in, + })) + require.Nil(t, err) + require.Equal(t, in, series.Values) + } + + // single input -> same output + singleSeries := []*ts.Series{consolidationTestSeries[0]} + r, err = f(ctx, multiplePathSpecs(ts.SeriesList{ + Values: singleSeries, + })) + require.Nil(t, err) + + series = r.Values + require.Equal(t, singleSeries[0].Len(), series[0].Len()) + for i := 0; i < series[0].Len(); i++ { + assert.Equal(t, singleSeries[0].ValueAt(i), series[0].ValueAt(i)) + } +} + +func TestMinSeries(t *testing.T) { + testAggregatedSeries(t, minSeries, 15.0, 3.0, 3.0, 17.0, "invalid min value for step %d") +} + +func TestMaxSeries(t *testing.T) { + testAggregatedSeries(t, maxSeries, 15.0, 15.0, 17.0, 17.0, "invalid max value for step %d") +} + +func TestSumSeries(t *testing.T) { + testAggregatedSeries(t, func(ctx *common.Context, series multiplePathSpecs) (ts.SeriesList, error) { + return sumSeries(ctx, series) + }, 15.0, 28.0, 30.0, 17.0, "invalid sum value for step %d") +} + +type mockEngine struct { + fn func( + ctx context.Context, + query string, + start, end time.Time, + timeout time.Duration, + ) (*storage.FetchResult, error) +} + +func (e mockEngine) FetchByQuery( + ctx context.Context, + query string, + start, end time.Time, + timeout time.Duration, +) (*storage.FetchResult, error) { + return e.fn(ctx, query, start, end, timeout) +} + +func TestVariadicSumSeries(t *testing.T) { + expr, err := compile("sumSeries(foo.bar.*, foo.baz.*)") + require.NoError(t, err) + + ctx := common.NewTestContext() + ctx.Engine = mockEngine{fn: func( + ctx context.Context, + query string, + start, end time.Time, + timeout time.Duration, + ) (*storage.FetchResult, error) { + switch query { + case "foo.bar.*": + return storage.NewFetchResult(ctx, []*ts.Series{ + ts.NewSeries(ctx, "foo.bar.a", start, ts.NewConstantValues(ctx, 1, 3, 1000)), + ts.NewSeries(ctx, "foo.bar.b", start, ts.NewConstantValues(ctx, 2, 3, 1000)), + }), nil + case "foo.baz.*": + return storage.NewFetchResult(ctx, []*ts.Series{ + ts.NewSeries(ctx, "foo.baz.a", start, ts.NewConstantValues(ctx, 3, 3, 1000)), + ts.NewSeries(ctx, "foo.baz.b", start, ts.NewConstantValues(ctx, 4, 3, 1000)), + }), nil + } + return nil, fmt.Errorf("unexpected query: %s", query) + }} + + r, err := expr.Execute(ctx) + require.NoError(t, err) + + require.Equal(t, 1, r.Len()) + assert.Equal(t, []float64{10, 10, 10}, r.Values[0].SafeValues()) +} + +func TestDiffSeries(t *testing.T) { + testAggregatedSeries(t, diffSeries, -15.0, -8.0, -10.0, -17.0, "invalid diff value for step %d") +} + +func TestMultiplySeries(t *testing.T) { + testAggregatedSeries(t, multiplySeries, 15.0, 450.0, 510.0, 17.0, "invalid product value for step %d") +} + +func TestAverageSeries(t *testing.T) { + testAggregatedSeries(t, averageSeries, 15.0, 28.0/3, 10.0, 17.0, "invalid avg value for step %d") +} + +func TestDivideSeries(t *testing.T) { + ctx, consolidationTestSeries := newConsolidationTestSeries() + defer ctx.Close() + + // multiple series, different start/end times + nan := math.NaN() + series, err := divideSeries(ctx, singlePathSpec{ + Values: consolidationTestSeries[0:2], + }, singlePathSpec{ + Values: consolidationTestSeries[2:3], + }) + require.Nil(t, err) + expected := []common.TestSeries{ + { + Name: "divideSeries(a,c)", + Data: []float64{nan, nan, nan, 0.5882, 0.5882, 0.5882, nan, nan, nan}, + }, + { + Name: "divideSeries(b,c)", + Data: []float64{nan, nan, nan, nan, nan, nan, nan, nan, nan, nan, nan, nan}, + }, + } + + common.CompareOutputsAndExpected(t, 10000, consolidationStartTime, + []common.TestSeries{expected[0]}, []*ts.Series{series.Values[0]}) + common.CompareOutputsAndExpected(t, 10000, consolidationStartTime.Add(-30*time.Second), + []common.TestSeries{expected[1]}, []*ts.Series{series.Values[1]}) + + // different millisPerStep, same start/end times + series, err = divideSeries(ctx, singlePathSpec{ + Values: consolidationTestSeries[0:1], + }, singlePathSpec{ + Values: consolidationTestSeries[3:4], + }) + require.Nil(t, err) + expected = []common.TestSeries{ + { + Name: "divideSeries(a,d)", + Data: []float64{3.3333, 3.3333, 3.3333, 3.3333, 3.33333, 3.3333}, + }, + } + common.CompareOutputsAndExpected(t, 10000, consolidationStartTime, + []common.TestSeries{expected[0]}, []*ts.Series{series.Values[0]}) + + // error - multiple divisor series + series, err = divideSeries(ctx, singlePathSpec{ + Values: consolidationTestSeries, + }, singlePathSpec{ + Values: consolidationTestSeries, + }) + require.Error(t, err) +} + +func TestAverageSeriesWithWildcards(t *testing.T) { + ctx, _ := newConsolidationTestSeries() + defer ctx.Close() + + input := []common.TestSeries{ + common.TestSeries{"web.host-1.avg-response.value", []float64{70.0, 20.0, 30.0, 40.0, 50.0}}, + common.TestSeries{"web.host-2.avg-response.value", []float64{20.0, 30.0, 40.0, 50.0, 60.0}}, + common.TestSeries{"web.host-3.avg-response.value", []float64{30.0, 40.0, 80.0, 60.0, 70.0}}, + common.TestSeries{"web.host-4.num-requests.value", []float64{10.0, 10.0, 15.0, 10.0, 15.0}}, + } + expected := []common.TestSeries{ + common.TestSeries{"web.avg-response", []float64{40.0, 30.0, 50.0, 50.0, 60.0}}, + common.TestSeries{"web.num-requests", []float64{10.0, 10.0, 15.0, 10.0, 15.0}}, + } + + start := consolidationStartTime + step := 12000 + timeSeries := generateSeriesList(ctx, start, input, step) + output, err := averageSeriesWithWildcards(ctx, singlePathSpec{ + Values: timeSeries, + }, 1, 3) + require.NoError(t, err) + sort.Sort(TimeSeriesPtrVector(output.Values)) + common.CompareOutputsAndExpected(t, step, start, expected, output.Values) +} + +func TestSumSeriesWithWildcards(t *testing.T) { + var ( + start, _ = time.Parse(time.RFC1123, "Mon, 27 Jul 2015 19:41:19 GMT") + end, _ = time.Parse(time.RFC1123, "Mon, 27 Jul 2015 19:43:19 GMT") + ctx = common.NewContext(common.ContextOptions{Start: start, End: end}) + inputs = []*ts.Series{ + ts.NewSeries(ctx, "servers.foo-1.pod1.status.500", start, + ts.NewConstantValues(ctx, 2, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-2.pod1.status.500", start, + ts.NewConstantValues(ctx, 4, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-3.pod1.status.500", start, + ts.NewConstantValues(ctx, 6, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-1.pod2.status.500", start, + ts.NewConstantValues(ctx, 8, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-2.pod2.status.500", start, + ts.NewConstantValues(ctx, 10, 12, 10000)), + + ts.NewSeries(ctx, "servers.foo-1.pod1.status.400", start, + ts.NewConstantValues(ctx, 20, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-2.pod1.status.400", start, + ts.NewConstantValues(ctx, 30, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-3.pod2.status.400", start, + ts.NewConstantValues(ctx, 40, 12, 10000)), + } + ) + defer ctx.Close() + + outSeries, err := sumSeriesWithWildcards(ctx, singlePathSpec{ + Values: inputs, + }, 1, 2) + require.NoError(t, err) + require.Equal(t, 2, len(outSeries.Values)) + + outSeries, _ = sortByName(ctx, singlePathSpec(outSeries)) + + expectedOutputs := []struct { + name string + sumOfVals float64 + }{ + {"servers.status.400", 90 * 12}, + {"servers.status.500", 30 * 12}, + } + + for i, expected := range expectedOutputs { + series := outSeries.Values[i] + assert.Equal(t, expected.name, series.Name()) + assert.Equal(t, expected.sumOfVals, series.SafeSum()) + } +} + +func TestGroupByNode(t *testing.T) { + var ( + start, _ = time.Parse(time.RFC1123, "Mon, 27 Jul 2015 19:41:19 GMT") + end, _ = time.Parse(time.RFC1123, "Mon, 27 Jul 2015 19:43:19 GMT") + ctx = common.NewContext(common.ContextOptions{Start: start, End: end}) + inputs = []*ts.Series{ + ts.NewSeries(ctx, "servers.foo-1.pod1.status.500", start, + ts.NewConstantValues(ctx, 2, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-2.pod1.status.500", start, + ts.NewConstantValues(ctx, 4, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-3.pod1.status.500", start, + ts.NewConstantValues(ctx, 6, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-1.pod2.status.500", start, + ts.NewConstantValues(ctx, 8, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-2.pod2.status.500", start, + ts.NewConstantValues(ctx, 10, 12, 10000)), + + ts.NewSeries(ctx, "servers.foo-1.pod1.status.400", start, + ts.NewConstantValues(ctx, 20, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-2.pod1.status.400", start, + ts.NewConstantValues(ctx, 30, 12, 10000)), + ts.NewSeries(ctx, "servers.foo-3.pod2.status.400", start, + ts.NewConstantValues(ctx, 40, 12, 10000)), + } + ) + defer ctx.Close() + + type result struct { + name string + sumOfVals float64 + } + + tests := []struct { + fname string + node int + expectedResults []result + }{ + {"avg", 4, []result{ + {"400", ((20 + 30 + 40) / 3) * 12}, + {"500", ((2 + 4 + 6 + 8 + 10) / 5) * 12}, + }}, + {"max", 2, []result{ + {"pod1", 30 * 12}, + {"pod2", 40 * 12}, + }}, + {"min", -1, []result{ + {"400", 20 * 12}, + {"500", 2 * 12}, + }}, + } + + for _, test := range tests { + outSeries, err := groupByNode(ctx, singlePathSpec{ + Values: inputs, + }, test.node, test.fname) + require.NoError(t, err) + require.Equal(t, len(test.expectedResults), len(outSeries.Values)) + + outSeries, _ = sortByName(ctx, singlePathSpec(outSeries)) + + for i, expected := range test.expectedResults { + series := outSeries.Values[i] + assert.Equal(t, expected.name, series.Name(), + "wrong name for %d %s (%d)", test.node, test.fname, i) + assert.Equal(t, expected.sumOfVals, series.SafeSum(), + "wrong result for %d %s (%d)", test.node, test.fname, i) + } + } +} + +func TestWeightedAverage(t *testing.T) { + ctx, _ := newConsolidationTestSeries() + defer ctx.Close() + + means := []common.TestSeries{ + common.TestSeries{"web.host-1.avg-response.mean", []float64{70.0, 20.0, 30.0, 0.0, 50.0}}, + common.TestSeries{"web.host-2.avg-response.mean", []float64{20.0, 30.0, 40.0, 50.0, 60.0}}, + common.TestSeries{"web.host-3.avg-response.mean", []float64{20.0, 30.0, 40.0, 50.0, 60.0}}, // no match + } + counts := []common.TestSeries{ + common.TestSeries{"web.host-1.avg-response.count", []float64{1, 2, 3, 4, 5}}, + common.TestSeries{"web.host-2.avg-response.count", []float64{10, 20, 30, 40, 50}}, + common.TestSeries{"web.host-4.avg-response.count", []float64{10, 20, 30, 40, 50}}, // no match + } + expected := []common.TestSeries{ + common.TestSeries{"weightedAverage", []float64{24.5454, 29.0909, 39.0909, 45.4545, 59.0909}}, + } + + // normal series + start := consolidationStartTime + step := 12000 + values := ts.SeriesList{Values: generateSeriesList(ctx, start, means, step)} + weights := ts.SeriesList{Values: generateSeriesList(ctx, start, counts, step)} + output, err := weightedAverage(ctx, singlePathSpec(values), singlePathSpec(weights), 1) + require.NoError(t, err) + sort.Sort(TimeSeriesPtrVector(output.Values)) + common.CompareOutputsAndExpected(t, step, start, expected, output.Values) + + // one series as input, should return the same as output no matter what the weight + values = ts.SeriesList{Values: generateSeriesList(ctx, start, means[:1], step)} + weights = ts.SeriesList{Values: generateSeriesList(ctx, start, counts[:1], step)} + output, err = weightedAverage(ctx, singlePathSpec(values), singlePathSpec(weights), 1) + require.NoError(t, err) + common.CompareOutputsAndExpected(t, step, start, + []common.TestSeries{{"weightedAverage", means[0].Data}}, output.Values) + + // different steps should lead to error -- not supported yet + values = ts.SeriesList{Values: generateSeriesList(ctx, start, means, step)} + weights = ts.SeriesList{Values: generateSeriesList(ctx, start, counts, step*2)} + output, err = weightedAverage(ctx, singlePathSpec(values), singlePathSpec(weights), 1) + require.EqualError(t, err, "different step sizes in input series not supported") +} + +func TestCountSeries(t *testing.T) { + ctx, input := newConsolidationTestSeries() + defer ctx.Close() + + results, err := countSeries(ctx, multiplePathSpecs(ts.SeriesList{ + Values: input, + })) + expected := common.TestSeries{ + Name: "countSeries(a,b,c,d)", + Data: []float64{4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, + } + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input[1].MillisPerStep(), input[1].StartTime(), + []common.TestSeries{expected}, results.Values) +} diff --git a/src/query/graphite/native/alias_functions.go b/src/query/graphite/native/alias_functions.go new file mode 100644 index 0000000000..4f33c1a1bc --- /dev/null +++ b/src/query/graphite/native/alias_functions.go @@ -0,0 +1,49 @@ +// Copyright (c) 2019 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 native + +import ( + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/ts" +) + +// alias takes one metric or a wildcard seriesList and a string in quotes. +// Prints the string instead of the metric name in the legend. +func alias(ctx *common.Context, series singlePathSpec, a string) (ts.SeriesList, error) { + return common.Alias(ctx, ts.SeriesList(series), a) +} + +// aliasByMetric takes a seriesList and applies an alias derived from the base +// metric name. +func aliasByMetric(ctx *common.Context, series singlePathSpec) (ts.SeriesList, error) { + return common.AliasByMetric(ctx, ts.SeriesList(series)) +} + +// aliasByNode renames a time series result according to a subset of the nodes +// in its hierarchy. +func aliasByNode(ctx *common.Context, seriesList singlePathSpec, nodes ...int) (ts.SeriesList, error) { + return common.AliasByNode(ctx, ts.SeriesList(seriesList), nodes...) +} + +// aliasSub runs series names through a regex search/replace. +func aliasSub(ctx *common.Context, input singlePathSpec, search, replace string) (ts.SeriesList, error) { + return common.AliasSub(ctx, ts.SeriesList(input), search, replace) +} diff --git a/src/query/graphite/native/alias_functions_test.go b/src/query/graphite/native/alias_functions_test.go new file mode 100644 index 0000000000..719444f5c7 --- /dev/null +++ b/src/query/graphite/native/alias_functions_test.go @@ -0,0 +1,219 @@ +// Copyright (c) 2019 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 native + +import ( + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestAlias(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + series := []*ts.Series{ + ts.NewSeries(ctx, "bender", now, values), + ts.NewSeries(ctx, "fry", now, values), + ts.NewSeries(ctx, "leela", now, values), + } + a := "farnsworth" + + results, err := alias(nil, singlePathSpec{ + Values: series, + }, a) + require.Nil(t, err) + require.NotNil(t, results) + require.Equal(t, len(series), results.Len()) + for _, s := range results.Values { + assert.Equal(t, a, s.Name()) + } +} + +func TestAliasSubWithNoBackReference(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + series := []*ts.Series{ + ts.NewSeries(ctx, "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.quux.john.Frank--submit", now, values), + ts.NewSeries(ctx, "stats.foo.counts.bar.baz.quail15-foo.qux.qaz.calls.success.quux.bob.BOB--nosub", now, values), + ts.NewSeries(ctx, "stats.foo.counts.bar.baz.quail01-foo.qux.qaz.calls.success.quux.bob.BOB--woop", now, values), + ts.NewSeries(ctx, "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.quacks.john.Frank--submit", now, values), + ts.NewSeries(ctx, "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.arbiter", now, values), // doesn't match regex + } + + results, err := aliasSub(ctx, singlePathSpec{ + Values: series, + }, "success\\.([-_\\w]+)\\.([-_\\w]+)\\.([-_\\w]+)", "$1-$3") + require.NoError(t, err) + require.Equal(t, 5, results.Len()) + + var names, pathExpr []string + for _, s := range results.Values { + names = append(names, s.Name()) + pathExpr = append(pathExpr, s.Specification) + } + + assert.Equal(t, []string{ + "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.quux-Frank--submit", + "stats.foo.counts.bar.baz.quail15-foo.qux.qaz.calls.quux-BOB--nosub", + "stats.foo.counts.bar.baz.quail01-foo.qux.qaz.calls.quux-BOB--woop", + "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.quacks-Frank--submit", + "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.arbiter", // unchanged + }, names) + + // Path expressions should remain unchanged + assert.Equal(t, []string{ + "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.quux.john.Frank--submit", + "stats.foo.counts.bar.baz.quail15-foo.qux.qaz.calls.success.quux.bob.BOB--nosub", + "stats.foo.counts.bar.baz.quail01-foo.qux.qaz.calls.success.quux.bob.BOB--woop", + "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.quacks.john.Frank--submit", + "stats.foo.counts.bar.baz.quail08-foo.qux.qaz.calls.success.arbiter", + }, pathExpr) +} + +func TestAliasSubWithBackReferences(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := []float64{1.0, 2.0, 3.0, 4.0} + + input := struct { + name string + pattern string + replace string + expected string + }{ + "stats.foo.timers.qaz.qaz.views.quail.end_to_end_latency.p95", + `stats.(.*).timers.\w+.qaz.views.quail.(.*)`, + `\1.\2`, + "foo.end_to_end_latency.p95", + } + series := []*ts.Series{ts.NewSeries(ctx, input.name, now, common.NewTestSeriesValues(ctx, 1000, values))} + results, err := aliasSub(ctx, singlePathSpec{ + Values: series, + }, input.pattern, input.replace) + require.NoError(t, err) + expected := []common.TestSeries{ + common.TestSeries{Name: input.expected, Data: values}, + } + common.CompareOutputsAndExpected(t, 1000, now, expected, results.Values) + + results, err = aliasSub(ctx, singlePathSpec{ + Values: series, + }, input.pattern, `\1.\3`) + require.Error(t, err) + require.Nil(t, results.Values) +} + +func TestAliasByMetric(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + + series := []*ts.Series{ + ts.NewSeries(ctx, "foo.bar.baz.foo01-foo.writes.success", now, values), + ts.NewSeries(ctx, "foo.bar.baz.foo02-foo.writes.success.P99", now, values), + ts.NewSeries(ctx, "foo.bar.baz.foo03-foo.writes.success.P75", now, values), + ts.NewSeries(ctx, "scale(stats.foobar.gauges.quazqux.latency_minutes.foo, 60.123))", now, values), + } + + results, err := aliasByMetric(ctx, singlePathSpec{ + Values: series, + }) + require.Nil(t, err) + require.NotNil(t, results) + require.Equal(t, len(series), len(results.Values)) + assert.Equal(t, "success", results.Values[0].Name()) + assert.Equal(t, "P99", results.Values[1].Name()) + assert.Equal(t, "P75", results.Values[2].Name()) + assert.Equal(t, "foo", results.Values[3].Name()) +} + +func TestAliasByNode(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + + series := []*ts.Series{ + ts.NewSeries(ctx, "foo.bar.baz.foo01-foo.writes.success", now, values), + ts.NewSeries(ctx, "foo.bar.baz.foo02-foo.writes.success.P99", now, values), + ts.NewSeries(ctx, "foo.bar.baz.foo03-foo.writes.success.P75", now, values), + } + + results, err := aliasByNode(ctx, singlePathSpec{ + Values: series, + }, 3, 5, 6) + require.Nil(t, err) + require.NotNil(t, results) + require.Equal(t, len(series), results.Len()) + assert.Equal(t, "foo01-foo.success", results.Values[0].Name()) + assert.Equal(t, "foo02-foo.success.P99", results.Values[1].Name()) + assert.Equal(t, "foo03-foo.success.P75", results.Values[2].Name()) + + results, err = aliasByNode(nil, singlePathSpec{ + Values: series, + }, -1) + require.Nil(t, err) + require.NotNil(t, results) + require.Equal(t, len(series), results.Len()) + assert.Equal(t, "success", results.Values[0].Name()) + assert.Equal(t, "P99", results.Values[1].Name()) + assert.Equal(t, "P75", results.Values[2].Name()) +} + +func TestAliasByNodeWithComposition(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + series := []*ts.Series{ + ts.NewSeries(ctx, "derivative(servers.bob02-foo.cpu.load_5)", now, values), + ts.NewSeries(ctx, "derivative(derivative(servers.bob02-foo.cpu.load_5))", now, values), + ts.NewSeries(ctx, "~~~", now, values), + ts.NewSeries(ctx, "", now, values), + } + results, err := aliasByNode(ctx, singlePathSpec{ + Values: series, + }, 0, 1) + require.Nil(t, err) + require.NotNil(t, results) + require.Equal(t, len(series), results.Len()) + assert.Equal(t, "servers.bob02-foo", results.Values[0].Name()) + assert.Equal(t, "servers.bob02-foo", results.Values[1].Name()) + assert.Equal(t, "~~~", results.Values[2].Name()) + assert.Equal(t, "", results.Values[3].Name()) +} diff --git a/src/query/graphite/native/builtin_functions.go b/src/query/graphite/native/builtin_functions.go new file mode 100644 index 0000000000..52a683bfa7 --- /dev/null +++ b/src/query/graphite/native/builtin_functions.go @@ -0,0 +1,1951 @@ +// Copyright (c) 2019 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 native + +import ( + "bytes" + "fmt" + "math" + "math/rand" + "regexp" + "sort" + "strings" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +const ( + millisPerSecond = 1000 + secondsPerDay = 24 * 3600 + daysPerWeek = 7 + secondsPerWeek = secondsPerDay * daysPerWeek + cactiStyleFormat = "%.2f" + wrappingFmt = "%s(%s)" + alpha = 0.1 + gamma = 0.1 + beta = 0.0035 +) + +func joinPathExpr(series ts.SeriesList) string { + seen := make(map[string]struct{}) + + joined := make([]string, 0, series.Len()) + for _, s := range series.Values { + if len(s.Specification) == 0 { + continue + } + + if _, exists := seen[s.Specification]; exists { + continue + } + + seen[s.Specification] = struct{}{} + joined = append(joined, s.Specification) + } + + return strings.Join(joined, ",") +} + +// sortByName sorts timeseries results by their names +func sortByName(_ *common.Context, series singlePathSpec) (ts.SeriesList, error) { + sorted := make([]*ts.Series, len(series.Values)) + for i := range series.Values { + sorted[i] = series.Values[i] + } + + sort.Sort(ts.SeriesByName(sorted)) + + r := ts.SeriesList(series) + r.Values = sorted + r.SortApplied = true + return r, nil +} + +// sortByTotal sorts timeseries results by the sum of values. +func sortByTotal(ctx *common.Context, series singlePathSpec) (ts.SeriesList, error) { + return highestSum(ctx, series, len(series.Values)) +} + +// sortByMaxima sorts timeseries by the maximum value across the time period specified. +func sortByMaxima(ctx *common.Context, series singlePathSpec) (ts.SeriesList, error) { + return highestMax(ctx, series, len(series.Values)) +} + +type valueComparator func(v, threshold float64) bool + +func compareByFunction( + _ *common.Context, + series singlePathSpec, + sr ts.SeriesReducer, + vc valueComparator, + threshold float64, +) (ts.SeriesList, error) { + res := make([]*ts.Series, 0, len(series.Values)) + for _, s := range series.Values { + stats := sr(s) + if vc(stats, threshold) { + res = append(res, s) + } + } + + r := ts.SeriesList(series) + r.Values = res + return r, nil +} + +func aboveByFunction( + ctx *common.Context, + series singlePathSpec, + sr ts.SeriesReducer, + threshold float64, +) (ts.SeriesList, error) { + return compareByFunction(ctx, series, sr, func(stats, threshold float64) bool { + return stats > threshold + }, threshold) +} + +func belowByFunction( + ctx *common.Context, + series singlePathSpec, + sr ts.SeriesReducer, + threshold float64, +) (ts.SeriesList, error) { + return compareByFunction(ctx, series, sr, func(stats, threshold float64) bool { + return stats < threshold + }, threshold) +} + +// maximumAbove takes one metric or a wildcard seriesList followed by an floating point number n, +// returns only the metrics with a maximum value above n. +func maximumAbove(ctx *common.Context, series singlePathSpec, n float64) (ts.SeriesList, error) { + sr := ts.SeriesReducerMax.Reducer() + return aboveByFunction(ctx, series, sr, n) +} + +// minimumAbove takes one metric or a wildcard seriesList followed by an floating point number n, +// returns only the metrics with a minimum value above n. +func minimumAbove(ctx *common.Context, series singlePathSpec, n float64) (ts.SeriesList, error) { + sr := ts.SeriesReducerMin.Reducer() + return aboveByFunction(ctx, series, sr, n) +} + +// averageAbove takes one metric or a wildcard seriesList followed by an floating point number n, +// returns only the metrics with an average value above n. +func averageAbove(ctx *common.Context, series singlePathSpec, n float64) (ts.SeriesList, error) { + sr := ts.SeriesReducerAvg.Reducer() + return aboveByFunction(ctx, series, sr, n) +} + +// currentAbove takes one metric or a wildcard seriesList followed by an floating point number n, +// returns only the metrics with the last value above n. +func currentAbove(ctx *common.Context, series singlePathSpec, n float64) (ts.SeriesList, error) { + sr := ts.SeriesReducerLast.Reducer() + return aboveByFunction(ctx, series, sr, n) +} + +// currentBelow takes one metric or a wildcard seriesList followed by an floating point number n, +// returns only the metrics with the last value below n. +func currentBelow(ctx *common.Context, series singlePathSpec, n float64) (ts.SeriesList, error) { + sr := ts.SeriesReducerLast.Reducer() + return belowByFunction(ctx, series, sr, n) +} + +// constantLine takes value and creates a constant line at value. +func constantLine(ctx *common.Context, value float64) (ts.SeriesList, error) { + newSeries, err := common.ConstantLine(ctx, value) + if err != nil { + return ts.SeriesList{}, err + } + return ts.SeriesList{Values: []*ts.Series{newSeries}}, nil +} + +// identity returns datapoints where the value equals the timestamp of the datapoint. +func identity(ctx *common.Context, name string) (ts.SeriesList, error) { + return common.Identity(ctx, name) +} + +// limit takes one metric or a wildcard seriesList followed by an integer N, and draws +// the first N metrics. +func limit(_ *common.Context, series singlePathSpec, n int) (ts.SeriesList, error) { + if n < 0 { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("invalid limit parameter n: %d", n)) + } + upperBound := int(math.Min(float64(len(series.Values)), float64(n))) + + r := ts.SeriesList(series) + r.Values = series.Values[0:upperBound] + return r, nil +} + +// timeShift draws the selected metrics shifted in time. If no sign is given, a minus sign ( - ) is +// implied which will shift the metric back in time. If a plus sign ( + ) is given, the metric will +// be shifted forward in time +func timeShift( + _ *common.Context, + _ singlePathSpec, + timeShiftS string, + _ bool, +) (*unaryContextShifter, error) { + + // TODO: implement resetEnd + if !(strings.HasPrefix(timeShiftS, "+") || strings.HasPrefix(timeShiftS, "-")) { + timeShiftS = "-" + timeShiftS + } + + shift, err := common.ParseInterval(timeShiftS) + if err != nil { + return nil, errors.NewInvalidParamsError(fmt.Errorf("invalid timeShift parameter %s: %v", timeShiftS, err)) + } + + contextShiftingFn := func(c *common.Context) *common.Context { + opts := common.NewChildContextOptions() + opts.AdjustTimeRange(shift, shift, 0, 0) + childCtx := c.NewChildContext(opts) + return childCtx + } + + transformerFn := func(input ts.SeriesList) (ts.SeriesList, error) { + output := make([]*ts.Series, input.Len()) + for i, in := range input.Values { + // NB(jayp): opposite direction + output[i] = in.Shift(-1 * shift).RenamedTo(fmt.Sprintf("timeShift(%s, %s)", in.Name(), timeShiftS)) + } + input.Values = output + return input, nil + } + + return &unaryContextShifter{ + ContextShiftFunc: contextShiftingFn, + UnaryTransformer: transformerFn, + }, nil +} + +// absolute returns the absolute value of each element in the series. +func absolute(ctx *common.Context, input singlePathSpec) (ts.SeriesList, error) { + return transform(ctx, input, + func(fname string) string { return fmt.Sprintf(wrappingFmt, "absolute", fname) }, + math.Abs) +} + +// scale multiplies each element of a collection of time series by a given value +func scale(ctx *common.Context, input singlePathSpec, scale float64) (ts.SeriesList, error) { + return transform( + ctx, + input, + func(fname string) string { + newName := fmt.Sprintf("%s,"+common.FloatingPointFormat, fname, scale) + return fmt.Sprintf(wrappingFmt, "scale", newName) + }, + common.Scale(scale), + ) +} + +// scaleToSeconds makes a wildcard seriesList and returns "value per seconds" +func scaleToSeconds( + ctx *common.Context, + seriesList singlePathSpec, + seconds int, +) (ts.SeriesList, error) { + output := make([]*ts.Series, len(seriesList.Values)) + for i, series := range seriesList.Values { + var ( + outvals = ts.NewValues(ctx, series.MillisPerStep(), series.Len()) + name = fmt.Sprintf("scaleToSeconds(%s,%d)", series.Name(), seconds) + factor = float64(seconds*1000) / float64(series.MillisPerStep()) // convert seconds to millis + ) + for step := 0; step < series.Len(); step++ { + value := series.ValueAt(step) + outvals.SetValueAt(step, value*factor) + } + output[i] = ts.NewSeries(ctx, name, series.StartTime(), outvals) + } + + r := ts.SeriesList(seriesList) + r.Values = output + return r, nil +} + +// offset adds a value to each element of a collection of time series +func offset(ctx *common.Context, input singlePathSpec, factor float64) (ts.SeriesList, error) { + return transform( + ctx, + input, + func(fname string) string { + newName := fmt.Sprintf("%s,"+common.FloatingPointFormat, fname, factor) + return fmt.Sprintf(wrappingFmt, "offset", newName) + }, + common.Offset(factor), + ) +} + +// transform converts values in a timeseries according to the valueTransformer. +func transform(ctx *common.Context, input singlePathSpec, + fname func(inputName string) string, fn common.TransformFunc) (ts.SeriesList, error) { + + t := common.NewStatelessTransformer(fn) + return common.Transform(ctx, ts.SeriesList(input), t, func(in *ts.Series) string { + return fname(in.Name()) + }) +} + +// perSecond computes a derivative adjusted for the series time interval, +// useful for taking a running total metric and showing how many occurrences +// per second were handled +func perSecond(ctx *common.Context, input singlePathSpec, _ float64) (ts.SeriesList, error) { + // TODO: we are ignoring maxValue; we may need to implement it + return common.PerSecond(ctx, ts.SeriesList(input), func(series *ts.Series) string { + return fmt.Sprintf("perSecond(%s)", series.Name()) + }) +} + +// transformNull transforms all nulls (NaNs) in a time series to a defaultValue. +func transformNull(ctx *common.Context, input singlePathSpec, defaultValue float64) (ts.SeriesList, error) { + return transform( + ctx, + input, + func(fname string) string { + newName := fmt.Sprintf("%s,"+common.FloatingPointFormat, fname, defaultValue) + return fmt.Sprintf(wrappingFmt, "transformNull", newName) + }, + common.TransformNull(defaultValue), + ) +} + +// isNonNull takes a metric or wild card seriesList and counts up how many non-null values are specified. +// This is useful for understanding which metrics have data at a given point in time +// (ie, to count which servers are alive). +func isNonNull(ctx *common.Context, input singlePathSpec) (ts.SeriesList, error) { + return transform(ctx, + input, + func(fname string) string { return fmt.Sprintf(wrappingFmt, "isNonNull", fname) }, + common.IsNonNull()) +} + +// keepLastValue takes one metric or a wildcard seriesList, and optionally a limit to the number of +// NaN values to skip over. If not specified, limit has a default value of -1, meaning all NaNs will +// be replaced by the closest preceding value that's not an NaN. +func keepLastValue(ctx *common.Context, input singlePathSpec, limit int) (ts.SeriesList, error) { + output := make([]*ts.Series, 0, len(input.Values)) + for _, series := range input.Values { + consecutiveNaNs := 0 + numSteps := series.Len() + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + for i := 0; i < numSteps; i++ { + value := series.ValueAt(i) + vals.SetValueAt(i, value) + if i == 0 { + continue + } + if math.IsNaN(value) { + consecutiveNaNs++ + } else { + if limit == -1 || (consecutiveNaNs > 0 && consecutiveNaNs <= limit) { + v := series.ValueAt(i - consecutiveNaNs - 1) + if !math.IsNaN(v) { + for index := i - consecutiveNaNs; index < i; index++ { + vals.SetValueAt(index, v) + } + } + } + consecutiveNaNs = 0 + } + } + if limit == -1 || (consecutiveNaNs > 0 && consecutiveNaNs <= limit) { + for index := numSteps - consecutiveNaNs; index < numSteps; index++ { + vals.SetValueAt(index, series.ValueAt(numSteps-consecutiveNaNs-1)) + } + } + name := fmt.Sprintf("keepLastValue(%s)", series.Name()) + newSeries := ts.NewSeries(ctx, name, series.StartTime(), vals) + output = append(output, newSeries) + } + + r := ts.SeriesList(input) + r.Values = output + return r, nil +} + +type comparator func(float64, float64) bool + +// lessOrEqualFunc checks whether x is less than or equal to y +func lessOrEqualFunc(x float64, y float64) bool { + return x <= y +} + +// greaterOrEqualFunc checks whether x is greater or equal to y +func greaterOrEqualFunc(x float64, y float64) bool { + return x >= y +} + +func sustainedCompare(ctx *common.Context, input singlePathSpec, threshold float64, intervalString string, + comparisonFunction comparator, zeroValue float64, funcName string) (ts.SeriesList, error) { + output := make([]*ts.Series, 0, len(input.Values)) + interval, err := common.ParseInterval(intervalString) + if err != nil { + return ts.SeriesList{}, err + } + + intervalMillis := int(interval / time.Millisecond) + for _, series := range input.Values { + numSteps := series.Len() + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + + minSteps := intervalMillis / series.MillisPerStep() + currSteps := 0 + + for i := 0; i < numSteps; i++ { + value := series.ValueAt(i) + if comparisonFunction(value, threshold) { + currSteps++ + } else { + currSteps = 0 + } + if currSteps >= minSteps { + vals.SetValueAt(i, value) + } else { + vals.SetValueAt(i, zeroValue) + } + } + + name := fmt.Sprintf("%s(%s, %f, '%s')", + funcName, series.Name(), threshold, intervalString) + newSeries := ts.NewSeries(ctx, name, series.StartTime(), vals) + output = append(output, newSeries) + } + + r := ts.SeriesList(input) + r.Values = output + return r, nil +} + +func sustainedAbove(ctx *common.Context, input singlePathSpec, threshold float64, intervalString string) (ts.SeriesList, error) { + return sustainedCompare(ctx, input, threshold, intervalString, greaterOrEqualFunc, threshold-math.Abs(threshold), "sustainedAbove") +} + +func sustainedBelow(ctx *common.Context, input singlePathSpec, threshold float64, intervalString string) (ts.SeriesList, error) { + return sustainedCompare(ctx, input, threshold, intervalString, lessOrEqualFunc, threshold+math.Abs(threshold), "sustainedBelow") +} + +// removeBelowValue removes data below the given threshold from the series or list of series provided. +// Values below this threshold are assigned a value of None. +func removeBelowValue(ctx *common.Context, input singlePathSpec, n float64) (ts.SeriesList, error) { + return transform(ctx, input, + func(inputName string) string { + return fmt.Sprintf("removeBelowValue(%s, "+common.FloatingPointFormat+")", inputName, n) + }, + common.Filter(func(v float64) bool { return v >= n })) +} + +// removeAboveValue removes data above the given threshold from the series or list of series provided. +// Values above this threshold are assigned a value of None. +func removeAboveValue(ctx *common.Context, input singlePathSpec, n float64) (ts.SeriesList, error) { + return transform(ctx, input, + func(inputName string) string { + return fmt.Sprintf("removeAboveValue(%s, "+common.FloatingPointFormat+")", inputName, n) + }, + common.Filter(func(v float64) bool { return v <= n })) +} + +// removeEmptySeries returns only the time-series with non-empty data +func removeEmptySeries(ctx *common.Context, input singlePathSpec) (ts.SeriesList, error) { + return common.RemoveEmpty(ctx, ts.SeriesList(input)) +} + +func takeByFunction(input singlePathSpec, n int, sr ts.SeriesReducer, sort ts.Direction) (ts.SeriesList, error) { + series, err := ts.SortSeries(input.Values, sr, sort) + if err != nil { + return ts.SeriesList{}, err + } + r := ts.SeriesList{ + Values: series, + SortApplied: true, + } + return common.Head(r, n) +} + +// highestSum takes one metric or a wildcard seriesList followed by an integer +// n. Out of all metrics passed, draws only the N metrics with the highest +// total value in the time period specified. +func highestSum(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerSum.Reducer() + return takeByFunction(input, n, sr, ts.Descending) +} + +// highestMax takes one metric or a wildcard seriesList followed by an integer +// n. Out of all metrics passed, draws only the N metrics with the highest +// maximum value in the time period specified. +func highestMax(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerMax.Reducer() + return takeByFunction(input, n, sr, ts.Descending) +} + +// highestCurrent takes one metric or a wildcard seriesList followed by an +// integer n. Out of all metrics passed, draws only the N metrics with the +// highest value at the end of the time period specified. +func highestCurrent(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerLast.Reducer() + return takeByFunction(input, n, sr, ts.Descending) +} + +// highestAverage takes one metric or a wildcard seriesList followed by an +// integer n. Out of all metrics passed, draws only the top N metrics with the +// highest average value for the time period specified. +func highestAverage(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerAvg.Reducer() + return takeByFunction(input, n, sr, ts.Descending) +} + +// fallbackSeries takes one metric or a wildcard seriesList, and a second fallback metric. +// If the wildcard does not match any series, draws the fallback metric. +func fallbackSeries(_ *common.Context, input singlePathSpec, fallback singlePathSpec) (ts.SeriesList, error) { + if len(input.Values) > 0 { + return ts.SeriesList(input), nil + } + + return ts.SeriesList(fallback), nil +} + +// mostDeviant takes one metric or a wildcard seriesList followed by an integer +// N. Draws the N most deviant metrics. To find the deviants, the standard +// deviation (sigma) of each series is taken and ranked. The top N standard +// deviations are returned. +func mostDeviant(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerStdDev.Reducer() + return takeByFunction(input, n, sr, ts.Descending) +} + +// lowestAverage takes one metric or a wildcard seriesList followed by an +// integer n. Out of all metrics passed, draws only the top N metrics with the +// lowest average value for the time period specified. +func lowestAverage(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerAvg.Reducer() + return takeByFunction(input, n, sr, ts.Ascending) +} + +// lowestCurrent takes one metric or a wildcard seriesList followed by an +// integer n. Out of all metrics passed, draws only the N metrics with the +// lowest value at the end of the time period specified. +func lowestCurrent(_ *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) { + sr := ts.SeriesReducerLast.Reducer() + return takeByFunction(input, n, sr, ts.Ascending) +} + +// windowSizeFunc calculates window size for moving average calculation +type windowSizeFunc func(stepSize int) int + +// movingAverage calculates the moving average of a metric (or metrics) over a time interval. +func movingAverage(ctx *common.Context, input singlePathSpec, windowSizeValue genericInterface) (*binaryContextShifter, error) { + if len(input.Values) == 0 { + return nil, nil + } + + var delta time.Duration + var wf windowSizeFunc + var ws string + + switch windowSizeValue := windowSizeValue.(type) { + case string: + interval, err := common.ParseInterval(windowSizeValue) + if err != nil { + return nil, err + } + if interval <= 0 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "windowSize must be positive but instead is %v", + interval)) + return nil, err + } + wf = func(stepSize int) int { return int(int64(delta/time.Millisecond) / int64(stepSize)) } + ws = fmt.Sprintf("%q", windowSizeValue) + delta = interval + case float64: + windowSizeInt := int(windowSizeValue) + if windowSizeInt <= 0 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "windowSize must be positive but instead is %d", + windowSizeInt)) + return nil, err + } + wf = func(_ int) int { return windowSizeInt } + ws = fmt.Sprintf("%d", windowSizeInt) + maxStepSize := input.Values[0].MillisPerStep() + for i := 1; i < len(input.Values); i++ { + maxStepSize = int(math.Max(float64(maxStepSize), float64(input.Values[i].MillisPerStep()))) + } + delta = time.Duration(maxStepSize*windowSizeInt) * time.Millisecond + default: + err := errors.NewInvalidParamsError(fmt.Errorf( + "windowSize must be either a string or an int but instead is a %T", + windowSizeValue)) + return nil, err + } + + contextShiftingFn := func(c *common.Context) *common.Context { + opts := common.NewChildContextOptions() + opts.AdjustTimeRange(0, 0, delta, 0) + childCtx := c.NewChildContext(opts) + return childCtx + } + + bootstrapStartTime, bootstrapEndTime := ctx.StartTime.Add(-delta), ctx.StartTime + transformerFn := func(bootstrapped, original ts.SeriesList) (ts.SeriesList, error) { + bootstrapList, err := combineBootstrapWithOriginal(ctx, + bootstrapStartTime, bootstrapEndTime, + bootstrapped, singlePathSpec(original)) + if err != nil { + return ts.SeriesList{}, err + } + + results := make([]*ts.Series, 0, original.Len()) + for i, bootstrap := range bootstrapList.Values { + series := original.Values[i] + stepSize := series.MillisPerStep() + windowPoints := wf(stepSize) + if windowPoints == 0 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "windowSize should not be smaller than stepSize, windowSize=%v, stepSize=%d", + windowSizeValue, stepSize)) + return ts.SeriesList{}, err + } + + numSteps := series.Len() + offset := bootstrap.Len() - numSteps + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + sum := 0.0 + num := 0 + for i := 0; i < numSteps; i++ { + // skip if the number of points received is less than the number of points + // in the lookback window. + if offset < windowPoints { + continue + } + if i == 0 { + for j := offset - windowPoints; j < offset; j++ { + v := bootstrap.ValueAt(j) + if !math.IsNaN(v) { + sum += v + num++ + } + } + } else { + prev := bootstrap.ValueAt(i + offset - windowPoints - 1) + next := bootstrap.ValueAt(i + offset - 1) + if !math.IsNaN(prev) { + sum -= prev + num-- + } + if !math.IsNaN(next) { + sum += next + num++ + } + } + if num > 0 { + vals.SetValueAt(i, sum/float64(num)) + } + } + name := fmt.Sprintf("movingAverage(%s,%s)", series.Name(), ws) + newSeries := ts.NewSeries(ctx, name, series.StartTime(), vals) + results = append(results, newSeries) + } + + original.Values = results + return original, nil + } + + return &binaryContextShifter{ + ContextShiftFunc: contextShiftingFn, + BinaryTransformer: transformerFn, + }, nil +} + +// totalFunc takes an index and returns a total value for that index +type totalFunc func(int) float64 + +func totalBySum(seriesList []*ts.Series, index int) float64 { + s, hasValue := 0.0, false + for _, series := range seriesList { + v := series.ValueAt(index) + if !math.IsNaN(v) { + hasValue = true + s += v + } + } + if hasValue { + return s + } + return math.NaN() +} + +// asPercent calculates a percentage of the total of a wildcard series. +func asPercent(ctx *common.Context, input singlePathSpec, total genericInterface) (ts.SeriesList, error) { + if len(input.Values) == 0 { + return ts.SeriesList(input), nil + } + + var toNormalize, normalized []*ts.Series + var tf totalFunc + var totalText string + + switch totalArg := total.(type) { + case ts.SeriesList, singlePathSpec: + var total ts.SeriesList + switch v := totalArg.(type) { + case ts.SeriesList: + total = v + case singlePathSpec: + total = ts.SeriesList(v) + } + if total.Len() == 0 { + // normalize input and sum up input as the total series + toNormalize = input.Values + tf = func(idx int) float64 { return totalBySum(normalized, idx) } + } else { + // check total is a single-series list and normalize all of them + if total.Len() != 1 { + err := errors.NewInvalidParamsError(errors.New("total must be a single series")) + return ts.SeriesList{}, err + } + + toNormalize = append(input.Values, total.Values[0]) + tf = func(idx int) float64 { return normalized[len(normalized)-1].ValueAt(idx) } + totalText = total.Values[0].Name() + } + case float64: + toNormalize = input.Values + tf = func(idx int) float64 { return totalArg } + totalText = fmt.Sprintf(common.FloatingPointFormat, totalArg) + default: + err := errors.NewInvalidParamsError(errors.New("total is neither an int nor a series")) + return ts.SeriesList{}, err + } + + result, _, _, _, err := common.Normalize(ctx, ts.SeriesList{ + Values: toNormalize, + }) + if err != nil { + return ts.SeriesList{}, err + } + + normalized = result.Values + numInputSeries := len(input.Values) + values := make([]ts.MutableValues, 0, numInputSeries) + for i := 0; i < numInputSeries; i++ { + percents := ts.NewValues(ctx, normalized[i].MillisPerStep(), normalized[i].Len()) + values = append(values, percents) + } + for i := 0; i < normalized[0].Len(); i++ { + t := tf(i) + for j := 0; j < numInputSeries; j++ { + v := normalized[j].ValueAt(i) + if !math.IsNaN(v) && !math.IsNaN(t) && t != 0 { + values[j].SetValueAt(i, 100.0*v/t) + } + } + } + + results := make([]*ts.Series, 0, numInputSeries) + for i := 0; i < numInputSeries; i++ { + var totalName string + if len(totalText) == 0 { + totalName = normalized[i].Specification + } else { + totalName = totalText + } + newName := fmt.Sprintf("asPercent(%s, %s)", normalized[i].Name(), totalName) + newSeries := ts.NewSeries(ctx, newName, normalized[i].StartTime(), values[i]) + results = append(results, newSeries) + } + + r := ts.SeriesList(input) + r.Values = results + return r, nil +} + +// exclude takes a metric or a wildcard seriesList, followed by a regular +// expression in double quotes. Excludes metrics that match the regular +// expression. +func exclude(_ *common.Context, input singlePathSpec, pattern string) (ts.SeriesList, error) { + rePattern, err := regexp.Compile(pattern) + //NB(rooz): we decided to just fail if regex compilation fails to + //differentiate it from an all-excluding regex + if err != nil { + return ts.SeriesList{}, err + } + + output := make([]*ts.Series, 0, len(input.Values)) + for _, in := range input.Values { + if m := rePattern.FindStringSubmatch(strings.TrimSpace(in.Name())); len(m) == 0 { + output = append(output, in) + } + } + + r := ts.SeriesList(input) + r.Values = output + return r, nil +} + +// logarithm takes one metric or a wildcard seriesList, and draws the y-axis in +// logarithmic format. +func logarithm(ctx *common.Context, input singlePathSpec, base int) (ts.SeriesList, error) { + if base <= 0 { + err := errors.NewInvalidParamsError(fmt.Errorf("invalid log base %d", base)) + return ts.SeriesList{}, err + } + + results := make([]*ts.Series, 0, len(input.Values)) + for _, series := range input.Values { + vals := ts.NewValues(ctx, series.MillisPerStep(), series.Len()) + newName := fmt.Sprintf("log(%s, %d)", series.Name(), base) + if series.AllNaN() { + results = append(results, ts.NewSeries(ctx, newName, series.StartTime(), vals)) + continue + } + + for i := 0; i < series.Len(); i++ { + n := series.ValueAt(i) + if !math.IsNaN(n) && n > 0 { + vals.SetValueAt(i, math.Log10(n)/math.Log10(float64(base))) + } + } + + results = append(results, ts.NewSeries(ctx, newName, series.StartTime(), vals)) + } + + r := ts.SeriesList(input) + r.Values = results + return r, nil +} + +// group takes an arbitrary number of pathspecs and adds them to a single timeseries array. +// This function is used to pass multiple pathspecs to a function which only takes one +func group(_ *common.Context, input multiplePathSpecs) (ts.SeriesList, error) { + return ts.SeriesList(input), nil +} + +func derivativeTemplate(ctx *common.Context, input singlePathSpec, nameTemplate string, + fn func(float64, float64) float64) (ts.SeriesList, error) { + + output := make([]*ts.Series, len(input.Values)) + for i, in := range input.Values { + derivativeValues := ts.NewValues(ctx, in.MillisPerStep(), in.Len()) + previousValue := math.NaN() + + for step := 0; step < in.Len(); step++ { + value := in.ValueAt(step) + if math.IsNaN(value) || math.IsNaN(previousValue) { + derivativeValues.SetValueAt(step, math.NaN()) + } else { + derivativeValues.SetValueAt(step, fn(value, previousValue)) + } + + previousValue = value + } + + name := fmt.Sprintf("%s(%s)", nameTemplate, in.Name()) + output[i] = ts.NewSeries(ctx, name, in.StartTime(), derivativeValues) + } + + r := ts.SeriesList(input) + r.Values = output + return r, nil +} + +// integral shows the sum over time, sort of like a continuous addition function. +// Useful for finding totals or trends in metrics that are collected per minute. +func integral(ctx *common.Context, input singlePathSpec) (ts.SeriesList, error) { + results := make([]*ts.Series, 0, len(input.Values)) + for _, series := range input.Values { + if series.AllNaN() { + results = append(results, series) + continue + } + + outvals := ts.NewValues(ctx, series.MillisPerStep(), series.Len()) + var current float64 + for i := 0; i < series.Len(); i++ { + n := series.ValueAt(i) + if !math.IsNaN(n) { + current += n + outvals.SetValueAt(i, current) + } + } + + newName := fmt.Sprintf("integral(%s)", series.Name()) + results = append(results, ts.NewSeries(ctx, newName, series.StartTime(), outvals)) + } + + r := ts.SeriesList(input) + r.Values = results + return r, nil +} + +// This is the opposite of the integral function. This is useful for taking a +// running total metric and calculating the delta between subsequent data +// points. +func derivative(ctx *common.Context, input singlePathSpec) (ts.SeriesList, error) { + return derivativeTemplate(ctx, input, "derivative", func(value, previousValue float64) float64 { + return value - previousValue + }) +} + +// Same as the derivative function above, but ignores datapoints that trend down. +func nonNegativeDerivative(ctx *common.Context, input singlePathSpec, maxValue float64) (ts.SeriesList, error) { + return derivativeTemplate(ctx, input, "nonNegativeDerivative", func(value, previousValue float64) float64 { + difference := value - previousValue + if difference >= 0 { + return difference + } else if !math.IsNaN(maxValue) && maxValue >= value { + return (maxValue - previousValue) + value + 1.0 + } else { + return math.NaN() + } + }) +} + +// nPercentile returns percentile-percent of each series in the seriesList. +func nPercentile(ctx *common.Context, seriesList singlePathSpec, percentile float64) (ts.SeriesList, error) { + return common.NPercentile(ctx, ts.SeriesList(seriesList), percentile, func(name string, percentile float64) string { + return fmt.Sprintf("nPercentile(%s, "+common.FloatingPointFormat+")", name, percentile) + }) +} + +func percentileOfSeries(ctx *common.Context, seriesList singlePathSpec, percentile float64, interpolateValue genericInterface) (ts.SeriesList, error) { + if percentile <= 0 || percentile > 100 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "the requested percentile value must be betwween 0 and 100")) + return ts.SeriesList{}, err + } + + var interpolate bool + switch interpolateValue := interpolateValue.(type) { + case bool: + interpolate = interpolateValue + case string: + if interpolateValue == "true" { + interpolate = true + } else if interpolateValue != "false" { + err := errors.NewInvalidParamsError(fmt.Errorf( + "interpolateValue must be either true or false but instead is %s", + interpolateValue)) + return ts.SeriesList{}, err + } + default: + err := errors.NewInvalidParamsError(fmt.Errorf( + "interpolateValue must be either a boolean or a string but instead is %T", + interpolateValue)) + return ts.SeriesList{}, err + } + + if len(seriesList.Values) == 0 { + err := errors.NewInvalidParamsError(fmt.Errorf("series list cannot be empty")) + return ts.SeriesList{}, err + } + + normalize, _, _, _, err := common.Normalize(ctx, ts.SeriesList(seriesList)) + if err != nil { + return ts.SeriesList{}, err + } + + step := seriesList.Values[0].MillisPerStep() + for _, series := range seriesList.Values[1:] { + if step != series.MillisPerStep() { + err := errors.NewInvalidParamsError(fmt.Errorf( + "different step sizes in input series not supported")) + return ts.SeriesList{}, err + } + } + + // TODO: This is wrong when MillisPerStep is different across + // the timeseries. + min := seriesList.Values[0].Len() + for _, series := range seriesList.Values[1:] { + numSteps := series.Len() + if numSteps < min { + min = numSteps + } + } + + percentiles := make([]float64, min) + for i := 0; i < min; i++ { + row := make([]float64, len(seriesList.Values)) + for j, series := range seriesList.Values { + row[j] = series.ValueAt(i) + } + + percentiles[i] = common.GetPercentile(row, percentile, interpolate) + } + + percentilesSeries := ts.NewValues(ctx, normalize.Values[0].MillisPerStep(), min) + for k := 0; k < min; k++ { + percentilesSeries.SetValueAt(k, percentiles[k]) + } + + name := fmt.Sprintf("percentileOfSeries(%s,"+common.FloatingPointFormat+")", + seriesList.Values[0].Specification, percentile) + return ts.SeriesList{ + Values: []*ts.Series{ + ts.NewSeries(ctx, name, normalize.Values[0].StartTime(), percentilesSeries), + }, + }, nil +} + +// divMod takes dividend n and divisor m, returns quotient and remainder. +// prerequisite: m is nonzero. +func divMod(n, m int) (int, int) { + quotient := n / m + remainder := n - quotient*m + return quotient, remainder +} + +// addToBucket add value to buckets[idx] and handles NaNs properly. +func addToBucket(buckets ts.MutableValues, idx int, value float64) { + v := buckets.ValueAt(idx) + if math.IsNaN(v) { + buckets.SetValueAt(idx, value) + } else { + buckets.SetValueAt(idx, v+value) + } +} + +// durationToSeconds converts a duration to number of seconds. +func durationToSeconds(d time.Duration) int { + return int(d / time.Second) +} + +// hitcount estimates hit counts from a list of time series. This function assumes the values in each time +// series represent hits per second. It calculates hits per some larger interval such as per day or per hour. +// NB(xichen): it doesn't support the alignToInterval parameter because no one seems to be using that. +func hitcount(ctx *common.Context, seriesList singlePathSpec, intervalString string) (ts.SeriesList, error) { + interval, err := common.ParseInterval(intervalString) + if err != nil { + return ts.SeriesList{}, err + } + + intervalInSeconds := durationToSeconds(interval) + if intervalInSeconds <= 0 { + return ts.SeriesList{}, common.ErrInvalidIntervalFormat + } + + resultSeries := make([]*ts.Series, len(seriesList.Values)) + for index, series := range seriesList.Values { + step := time.Duration(series.MillisPerStep()) * time.Millisecond + bucketCount := int(math.Ceil(float64(series.EndTime().Sub(series.StartTime())) / float64(interval))) + buckets := ts.NewValues(ctx, int(interval/time.Millisecond), bucketCount) + newStart := series.EndTime().Add(-time.Duration(bucketCount) * interval) + + for i := 0; i < series.Len(); i++ { + value := series.ValueAt(i) + if math.IsNaN(value) { + continue + } + startTime := series.StartTime().Add(time.Duration(i) * step) + startBucket, startMod := divMod(durationToSeconds(startTime.Sub(newStart)), intervalInSeconds) + endTime := startTime.Add(step) + endBucket, endMod := divMod(durationToSeconds(endTime.Sub(newStart)), intervalInSeconds) + + if endBucket >= bucketCount { + endBucket = bucketCount - 1 + endMod = intervalInSeconds + } + + if startBucket == endBucket { + addToBucket(buckets, startBucket, value*float64(endMod-startMod)) + } else { + addToBucket(buckets, startBucket, value*float64(intervalInSeconds-startMod)) + hitsPerBucket := value * float64(intervalInSeconds) + for j := startBucket + 1; j < endBucket; j++ { + addToBucket(buckets, j, hitsPerBucket) + } + if endMod > 0 { + addToBucket(buckets, endBucket, value*float64(endMod)) + } + } + } + newName := fmt.Sprintf("hitcount(%s, %q)", series.Name(), intervalString) + newSeries := ts.NewSeries(ctx, newName, newStart, buckets) + resultSeries[index] = newSeries + } + + r := ts.SeriesList(seriesList) + r.Values = resultSeries + return r, nil +} + +func safeIndex(len, index int) int { + return int(math.Min(float64(index), float64(len))) +} + +// substr takes one metric or a wildcard seriesList followed by 1 or 2 integers. Prints n - length elements +// of the array (if only one integer n is passed) or n - m elements of the array (if two integers n and m +// are passed). +func substr(_ *common.Context, seriesList singlePathSpec, start, stop int) (ts.SeriesList, error) { + results := make([]*ts.Series, len(seriesList.Values)) + re := regexp.MustCompile(",.*$") + for i, series := range seriesList.Values { + name := series.Name() + left := strings.LastIndex(name, "(") + 1 + right := strings.Index(name, ")") + length := len(name) + if right < 0 { + right = length + } + right = safeIndex(length, right) + nameParts := strings.Split(name[left:right], ".") + numParts := len(nameParts) + // If stop == 0, it's as if stop was unspecified + if start < 0 || start >= numParts || (stop != 0 && stop < start) { + err := errors.NewInvalidParamsError(fmt.Errorf( + "invalid substr params, start=%d, stop=%d", start, stop)) + return ts.SeriesList{}, err + } + var newName string + if stop == 0 { + newName = strings.Join(nameParts[start:], ".") + } else { + stop = safeIndex(numParts, stop) + newName = strings.Join(nameParts[start:stop], ".") + } + newName = re.ReplaceAllString(newName, "") + results[i] = series.RenamedTo(newName) + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// combineBootstrapWithOriginal combines the bootstrapped the series with the original series. +func combineBootstrapWithOriginal( + ctx *common.Context, + startTime time.Time, + endTime time.Time, + bootstrapped ts.SeriesList, + seriesList singlePathSpec, +) (ts.SeriesList, error) { + nameToSeries := make(map[string]*ts.Series) + for _, series := range bootstrapped.Values { + nameToSeries[series.Name()] = series + } + + bootstrapList := make([]*ts.Series, 0, len(seriesList.Values)) + for _, series := range seriesList.Values { + bs, found := nameToSeries[series.Name()] + if !found { + numSteps := ts.NumSteps(startTime, endTime, series.MillisPerStep()) + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + bs = ts.NewSeries(ctx, series.Name(), startTime, vals) + } + bootstrapList = append(bootstrapList, bs) + } + + var err error + newSeriesList := make([]*ts.Series, len(seriesList.Values)) + for i, bootstrap := range bootstrapList { + original := seriesList.Values[i] + if bootstrap.MillisPerStep() < original.MillisPerStep() { + bootstrap, err = bootstrap.IntersectAndResize(bootstrap.StartTime(), bootstrap.EndTime(), original.MillisPerStep(), original.ConsolidationFunc()) + if err != nil { + return ts.SeriesList{}, err + } + } + // NB(braskin): using bootstrap.Len() is incorrect as it will include all + // of the steps in the original timeseries, not just the steps up to the new end time + bootstrapLength := bootstrap.StepAtTime(endTime) + ratio := bootstrap.MillisPerStep() / original.MillisPerStep() + numBootstrapValues := bootstrapLength * ratio + numCombinedValues := numBootstrapValues + original.Len() + values := ts.NewValues(ctx, original.MillisPerStep(), numCombinedValues) + for j := 0; j < min(bootstrap.Len(), bootstrapLength); j++ { + for k := j * ratio; k < (j+1)*ratio; k++ { + values.SetValueAt(k, bootstrap.ValueAt(j)) + } + } + for j := numBootstrapValues; j < numCombinedValues; j++ { + values.SetValueAt(j, original.ValueAt(j-numBootstrapValues)) + } + newSeries := ts.NewSeries(ctx, original.Name(), startTime, values) + newSeries.Specification = original.Specification + newSeriesList[i] = newSeries + } + + r := ts.SeriesList(seriesList) + r.Values = newSeriesList + return r, nil +} + +func min(a, b int) int { + if a < b { + return a + } + return b +} + +// trimBootstrap trims the bootstrap period off the front of this series so it matches the original. +func trimBootstrap(ctx *common.Context, bootstrap, original *ts.Series) *ts.Series { + originalLen := original.Len() + bootstrapLen := bootstrap.Len() + lengthLimit := (originalLen * original.MillisPerStep()) / bootstrap.MillisPerStep() + trimStart := bootstrap.EndTime().Add(-time.Duration(lengthLimit*bootstrap.MillisPerStep()) * time.Millisecond) + vals := ts.NewValues(ctx, bootstrap.MillisPerStep(), lengthLimit) + for i := 0; i < lengthLimit; i++ { + vals.SetValueAt(i, bootstrap.ValueAt(i+bootstrapLen-lengthLimit)) + } + return ts.NewSeries(ctx, bootstrap.Name(), trimStart, vals) +} + +// holtWintersForecast performs a Holt-Winters forecast using the series as input data. +// Data from one week previous to the series is used to bootstrap the initial forecast. +func holtWintersForecast(ctx *common.Context, seriesList singlePathSpec) (ts.SeriesList, error) { + return holtWintersForecastInternal(ctx, seriesList, secondsPerWeek*time.Second) +} + +// nolint: unparam +func holtWintersForecastInternal(ctx *common.Context, seriesList singlePathSpec, d time.Duration) (ts.SeriesList, error) { + results := make([]*ts.Series, len(seriesList.Values)) + bootstrapList, err := common.FetchWithBootstrap(ctx, ts.SeriesList(seriesList), d) + if err != nil { + return ts.SeriesList{}, err + } + for i, bootstrap := range bootstrapList.Values { + series := seriesList.Values[i] + analysis := holtWintersAnalysis(ctx, bootstrap) + results[i] = trimBootstrap(ctx, analysis.predictions, series) + } + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// holtWintersConfidenceBands performs a Holt-Winters forecast using the series as input data and +// plots upper and lower bands with the predicted forecast deviations. +func holtWintersConfidenceBands(ctx *common.Context, seriesList singlePathSpec, delta float64) (ts.SeriesList, error) { + return holtWintersConfidenceBandsInternal(ctx, seriesList, delta, secondsPerWeek*time.Second) +} + +// nolint: unparam +func holtWintersConfidenceBandsInternal(ctx *common.Context, seriesList singlePathSpec, delta float64, d time.Duration) (ts.SeriesList, error) { + results := make([]*ts.Series, 2*len(seriesList.Values)) + bootstrapList, err := common.FetchWithBootstrap(ctx, ts.SeriesList(seriesList), d) + if err != nil { + return ts.SeriesList{}, err + } + + for index, bootstrap := range bootstrapList.Values { + series := seriesList.Values[index] + analysis := holtWintersAnalysis(ctx, bootstrap) + forecast := trimBootstrap(ctx, analysis.predictions, series) + deviation := trimBootstrap(ctx, analysis.deviations, series) + seriesLength := forecast.Len() + upperBand := ts.NewValues(ctx, forecast.MillisPerStep(), seriesLength) + lowerBand := ts.NewValues(ctx, forecast.MillisPerStep(), seriesLength) + for i := 0; i < seriesLength; i++ { + forecastItem := forecast.ValueAt(i) + deviationItem := deviation.ValueAt(i) + if !math.IsNaN(forecastItem) && !math.IsNaN(deviationItem) { + scaledDeviation := delta * deviationItem + upperBand.SetValueAt(i, forecastItem+scaledDeviation) + lowerBand.SetValueAt(i, forecastItem-scaledDeviation) + } + } + upperName := fmt.Sprintf("holtWintersConfidenceUpper(%s)", series.Name()) + lowerName := fmt.Sprintf("holtWintersConfidenceLower(%s)", series.Name()) + upperSeries := ts.NewSeries(ctx, upperName, forecast.StartTime(), upperBand) + lowerSeries := ts.NewSeries(ctx, lowerName, forecast.StartTime(), lowerBand) + newIndex := index * 2 + results[newIndex] = lowerSeries + results[newIndex+1] = upperSeries + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// holtWintersAberration performs a Holt-Winters forecast using the series as input data and +// plots the positive or negative deviation of the series data from the forecast. +func holtWintersAberration(ctx *common.Context, seriesList singlePathSpec, delta float64) (ts.SeriesList, error) { + // log if we are actually using this function + var b bytes.Buffer + b.WriteString("holtWintersAberration is used") + if len(seriesList.Values) > 0 { + b.WriteString(fmt.Sprintf(", series[0] name=%s", seriesList.Values[0].Name())) + } + return holtWintersAberrationInternal(ctx, seriesList, delta, secondsPerWeek*time.Second) +} + +// nolint: unparam +func holtWintersAberrationInternal( + ctx *common.Context, + seriesList singlePathSpec, + delta float64, + d time.Duration, +) (ts.SeriesList, error) { + results := make([]*ts.Series, len(seriesList.Values)) + for index, series := range seriesList.Values { + confidenceBands, err := holtWintersConfidenceBandsInternal(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, delta, d) + if err != nil { + return ts.SeriesList{}, err + } + + lowerBand := confidenceBands.Values[0] + upperBand := confidenceBands.Values[1] + numPoints := series.Len() + aberration := ts.NewValues(ctx, series.MillisPerStep(), numPoints) + for i := 0; i < numPoints; i++ { + actual := series.ValueAt(i) + upperVal := upperBand.ValueAt(i) + lowerVal := lowerBand.ValueAt(i) + var newValue float64 + if math.IsNaN(actual) { + newValue = 0 + } else if !math.IsNaN(upperVal) && actual > upperVal { + newValue = actual - upperVal + } else if !math.IsNaN(lowerVal) && actual < lowerVal { + newValue = actual - lowerVal + } else { + newValue = 0 + } + aberration.SetValueAt(i, newValue) + } + newName := fmt.Sprintf("holtWintersAberration(%s)", series.Name()) + newSeries := ts.NewSeries(ctx, newName, series.StartTime(), aberration) + results[index] = newSeries + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +func holtWintersIntercept(actual, lastSeason, lastIntercept, lastSlope float64) float64 { + return alpha*(actual-lastSeason) + (1-alpha)*(lastIntercept+lastSlope) +} + +func holtWintersSlope(intercept, lastIntercept, lastSlope float64) float64 { + return beta*(intercept-lastIntercept) + (1-beta)*lastSlope +} + +func holtWintersSeasonal(actual, intercept, lastSeason float64) float64 { + return gamma*(actual-intercept) + (1-gamma)*lastSeason +} + +func holtWintersDeviation(actual, prediction, lastSeasonalDev float64) float64 { + if math.IsNaN(prediction) { + prediction = 0 + } + return gamma*math.Abs(actual-prediction) + (1-gamma)*lastSeasonalDev +} + +type holtWintersAnalysisResult struct { + predictions *ts.Series + deviations *ts.Series + intercepts []float64 + slopes []float64 + seasonals []float64 +} + +// holtWintersAnalysis takes a series, and returns the analysis result. +func holtWintersAnalysis(ctx *common.Context, series *ts.Series) *holtWintersAnalysisResult { + seasonLength := secondsPerDay * millisPerSecond / series.MillisPerStep() + numPoints := series.Len() + intercepts := make([]float64, numPoints) + slopes := make([]float64, numPoints) + seasonals := make([]float64, numPoints) + predictions := ts.NewValues(ctx, series.MillisPerStep(), numPoints) + deviations := ts.NewValues(ctx, series.MillisPerStep(), numPoints) + + getLastSeasonal := func(i int) float64 { + j := i - seasonLength + if j >= 0 { + return seasonals[j] + } + return 0.0 + } + + getLastDeviation := func(i int) float64 { + j := i - seasonLength + if j >= 0 { + return deviations.ValueAt(j) + } + return 0.0 + } + + nextPred := math.NaN() + for i := 0; i < numPoints; i++ { + actual := series.ValueAt(i) + if math.IsNaN(actual) { + intercepts[i] = math.NaN() + predictions.SetValueAt(i, nextPred) + deviations.SetValueAt(i, 0.0) + nextPred = math.NaN() + continue + } + + var lastIntercept, lastSlope, prediction float64 + if i == 0 { + lastIntercept = actual + lastSlope = 0 + prediction = actual + } else { + lastIntercept = intercepts[i-1] + lastSlope = slopes[i-1] + if math.IsNaN(lastIntercept) { + lastIntercept = actual + } + prediction = nextPred + } + + lastSeasonal := getLastSeasonal(i) + nextLastSeasonal := getLastSeasonal(i + 1) + lastSeasonalDev := getLastDeviation(i) + + intercept := holtWintersIntercept(actual, lastSeasonal, lastIntercept, lastSlope) + slope := holtWintersSlope(intercept, lastIntercept, lastSlope) + seasonal := holtWintersSeasonal(actual, intercept, lastSeasonal) + nextPred = intercept + slope + nextLastSeasonal + deviation := holtWintersDeviation(actual, prediction, lastSeasonalDev) + + intercepts[i] = intercept + slopes[i] = slope + seasonals[i] = seasonal + predictions.SetValueAt(i, prediction) + deviations.SetValueAt(i, deviation) + } + + // make the new forecast series + forecastName := fmt.Sprintf("holtWintersForecast(%s)", series.Name()) + forecastSeries := ts.NewSeries(ctx, forecastName, series.StartTime(), predictions) + + // make the new deviation series + deviationName := fmt.Sprintf("holtWintersDeviation(%s)", series.Name()) + deviationSeries := ts.NewSeries(ctx, deviationName, series.StartTime(), deviations) + + return &holtWintersAnalysisResult{ + predictions: forecastSeries, + deviations: deviationSeries, + intercepts: intercepts, + slopes: slopes, + seasonals: seasonals, + } +} + +// squareRoot takes one metric or a wildcard seriesList, and computes the square root of each datapoint. +func squareRoot(ctx *common.Context, seriesList singlePathSpec) (ts.SeriesList, error) { + return transform( + ctx, + seriesList, + func(fname string) string { return fmt.Sprintf(wrappingFmt, "squareRoot", fname) }, + math.Sqrt, + ) +} + +// stdev takes one metric or a wildcard seriesList followed by an integer N. Draw the standard deviation +// of all metrics passed for the past N datapoints. If the ratio of null points in the window is greater than +// windowTolerance, skip the calculation. +func stdev(ctx *common.Context, seriesList singlePathSpec, points int, windowTolerance float64) (ts.SeriesList, error) { + return common.Stdev(ctx, ts.SeriesList(seriesList), points, windowTolerance, func(series *ts.Series, points int) string { + return fmt.Sprintf("stddev(%s,%d)", series.Name(), points) + }) +} + +// rangeOfSeries distills down a set of inputs into the range of the series. +func rangeOfSeries(ctx *common.Context, seriesList singlePathSpec) (ts.SeriesList, error) { + series, err := common.Range(ctx, ts.SeriesList(seriesList), func(series ts.SeriesList) string { + return wrapPathExpr("rangeOfSeries", series) + }) + if err != nil { + return ts.SeriesList{}, err + } + return ts.SeriesList{Values: []*ts.Series{series}}, nil +} + +// removeAbovePercentile removes data above the specified percentile from the series +// or list of series provided. Values above this percentile are assigned a value +// of None. +func removeAbovePercentile(ctx *common.Context, seriesList singlePathSpec, percentile float64) (ts.SeriesList, error) { + return common.RemoveByPercentile(ctx, + ts.SeriesList(seriesList), + percentile, + func(name string, percentile float64) string { + return fmt.Sprintf("removeAbovePercentile(%s, "+common.FloatingPointFormat+")", name, percentile) + }, + common.GreaterThan) +} + +// removeBelowPercentile removes data below the specified percentile from the series +// or list of series provided. Values below this percentile are assigned a value of None. +func removeBelowPercentile(ctx *common.Context, seriesList singlePathSpec, percentile float64) (ts.SeriesList, error) { + return common.RemoveByPercentile( + ctx, + ts.SeriesList(seriesList), + percentile, + func(name string, percentile float64) string { + return fmt.Sprintf("removeBelowPercentile(%s, "+common.FloatingPointFormat+")", name, percentile) + }, + common.LessThan) +} + +// randomWalkFunction returns a random walk starting at 0. +// Note: step has a unit of seconds. +func randomWalkFunction(ctx *common.Context, name string, step int) (ts.SeriesList, error) { + if step <= 0 { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("non-positive step size %d", step)) + } + if !ctx.StartTime.Before(ctx.EndTime) { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("startTime %v is no earlier than endTime %v", ctx.StartTime, ctx.EndTime)) + } + r := rand.New(rand.NewSource(time.Now().UnixNano())) + millisPerStep := step * millisPerSecond + numSteps := ts.NumSteps(ctx.StartTime, ctx.EndTime, millisPerStep) + vals := ts.NewValues(ctx, millisPerStep, numSteps) + for i := 0; i < numSteps; i++ { + vals.SetValueAt(i, r.Float64()-0.5) + } + newSeries := ts.NewSeries(ctx, name, ctx.StartTime, vals) + return ts.SeriesList{Values: []*ts.Series{newSeries}}, nil +} + +// aggregateLine draws a horizontal line based the function applied to the series. +func aggregateLine(ctx *common.Context, seriesList singlePathSpec, f string) (ts.SeriesList, error) { + if len(seriesList.Values) == 0 { + return ts.SeriesList{}, common.ErrEmptySeriesList + } + + sa := ts.SeriesReducerApproach(f) + r, ok := sa.SafeReducer() + if !ok { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("invalid function %s", f)) + } + + value := r(seriesList.Values[0]) + name := fmt.Sprintf("aggregateLine(%s,"+common.FloatingPointFormat+")", + seriesList.Values[0].Specification, value) + series, err := constantLine(ctx, value) + if err != nil { + return ts.SeriesList{}, err + } + + renamed := series.Values[0].RenamedTo(name) + return ts.SeriesList{Values: []*ts.Series{renamed}}, nil +} + +// changed takes one metric or a wildcard seriesList. +// Output 1 when the value changed, 0 when null or the same. +func changed(ctx *common.Context, seriesList singlePathSpec) (ts.SeriesList, error) { + return common.Changed(ctx, ts.SeriesList(seriesList), func(series *ts.Series) string { + return fmt.Sprintf("changed(%s)", series.Name()) + }) +} + +// movingMedian takes one metric or a wildcard seriesList followed by a a quoted string +// with a length of time like '1hour' or '5min'. Graphs the median of the preceding +// datapoints for each point on the graph. All previous datapoints are set to None at +// the beginning of the graph. +func movingMedian(ctx *common.Context, _ singlePathSpec, windowSize string) (*binaryContextShifter, error) { + interval, err := common.ParseInterval(windowSize) + if err != nil { + return nil, err + } + if interval <= 0 { + return nil, common.ErrInvalidIntervalFormat + } + + contextShiftingFn := func(c *common.Context) *common.Context { + opts := common.NewChildContextOptions() + opts.AdjustTimeRange(0, 0, interval, 0) + childCtx := c.NewChildContext(opts) + return childCtx + } + + bootstrapStartTime, bootstrapEndTime := ctx.StartTime.Add(-interval), ctx.StartTime + transformerFn := func(bootstrapped, original ts.SeriesList) (ts.SeriesList, error) { + bootstrapList, err := combineBootstrapWithOriginal(ctx, + bootstrapStartTime, bootstrapEndTime, + bootstrapped, singlePathSpec(original)) + if err != nil { + return ts.SeriesList{}, err + } + + results := make([]*ts.Series, 0, original.Len()) + for i, bootstrap := range bootstrapList.Values { + series := original.Values[i] + windowPoints := int(interval / (time.Duration(series.MillisPerStep()) * time.Millisecond)) + if windowPoints <= 0 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "non positive window points, windowSize=%s, stepSize=%d", + windowSize, series.MillisPerStep())) + return ts.SeriesList{}, err + } + window := make([]float64, windowPoints) + numSteps := series.Len() + offset := bootstrap.Len() - numSteps + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + for i := 0; i < numSteps; i++ { + for j := i + offset - windowPoints; j < i+offset; j++ { + window[j-i-offset+windowPoints] = bootstrap.ValueAt(j) + } + nans := common.SafeSort(window) + if nans < windowPoints { + index := (windowPoints - nans) / 2 + median := window[nans+index] + vals.SetValueAt(i, median) + } + } + name := fmt.Sprintf("movingMedian(%s,%q)", series.Name(), windowSize) + newSeries := ts.NewSeries(ctx, name, series.StartTime(), vals) + results = append(results, newSeries) + } + + original.Values = results + return original, nil + } + + return &binaryContextShifter{ + ContextShiftFunc: contextShiftingFn, + BinaryTransformer: transformerFn, + }, nil +} + +// legendValue takes one metric or a wildcard seriesList and a string in quotes. +// Appends a value to the metric name in the legend. Currently one or several of: +// "last", "avg", "total", "min", "max". +func legendValue(_ *common.Context, seriesList singlePathSpec, valueType string) (ts.SeriesList, error) { + sa := ts.SeriesReducerApproach(valueType) + reducer, ok := sa.SafeReducer() + if !ok { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("invalid function %s", valueType)) + } + + results := make([]*ts.Series, 0, len(seriesList.Values)) + for _, series := range seriesList.Values { + value := reducer(series) + newName := fmt.Sprintf("%s (%s: "+common.FloatingPointFormat+")", series.Name(), valueType, value) + renamed := series.RenamedTo(newName) + results = append(results, renamed) + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +func getStatLen(v float64) int { + if math.IsNaN(v) { + return 4 + } + return len(fmt.Sprintf("%d", int(v))) + 3 +} + +func toCactiStyle(v float64) string { + if math.IsNaN(v) { + return "nan" + } + return fmt.Sprintf(cactiStyleFormat, v) +} + +func findAllLens(seriesList ts.SeriesList) (int, int, int, int) { + var nameLen, lastLen, maxLen, minLen float64 + for _, series := range seriesList.Values { + name, min, max, last := series.Name(), series.SafeMin(), series.SafeMax(), series.SafeLastValue() + nameLen = math.Max(nameLen, float64(len(name))) + lastLen = math.Max(lastLen, float64(getStatLen(last))) + maxLen = math.Max(maxLen, float64(getStatLen(max))) + minLen = math.Max(minLen, float64(getStatLen(min))) + } + return int(nameLen), int(lastLen) + 3, int(maxLen) + 3, int(minLen) + 3 +} + +// cactiStyle takes a series list and modifies the aliases to provide column aligned +// output with Current, Max, and Min values in the style of cacti. +func cactiStyle(_ *common.Context, seriesList singlePathSpec) (ts.SeriesList, error) { + if len(seriesList.Values) == 0 { + return ts.SeriesList{}, common.ErrEmptySeriesList + } + + nameLen, lastLen, maxLen, minLen := findAllLens(ts.SeriesList(seriesList)) + results := make([]*ts.Series, 0, len(seriesList.Values)) + for _, series := range seriesList.Values { + name := series.Name() + last := toCactiStyle(series.SafeLastValue()) + max := toCactiStyle(series.SafeMax()) + min := toCactiStyle(series.SafeMin()) + + newName := fmt.Sprintf( + "%*s Current:%*s Max:%*s Min:%*s ", + -nameLen, name, + -lastLen, last, + -maxLen, max, + -minLen, min, + ) + renamed := series.RenamedTo(newName) + results = append(results, renamed) + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// consolidateBy takes one metric or a wildcard seriesList and a consolidation +// function name. Valid function names are "sum", "average", "min", and "max". +// When a graph is drawn where width of the graph size in pixels is smaller than +// the number of data points to be graphed, m3 consolidates the values to +// to prevent line overlap. The consolidateBy() function changes the consolidation +// function from the default of "average" to one of "sum", "max", or "min". +func consolidateBy(_ *common.Context, seriesList singlePathSpec, consolidationApproach string) (ts.SeriesList, error) { + ca := ts.ConsolidationApproach(consolidationApproach) + cf, ok := ca.SafeFunc() + if !ok { + err := errors.NewInvalidParamsError(fmt.Errorf("invalid consolidation approach %s", consolidationApproach)) + return ts.SeriesList{}, err + } + + results := make([]*ts.Series, 0, len(seriesList.Values)) + for _, series := range seriesList.Values { + newName := fmt.Sprintf("consolidateBy(%s,%q)", series.Name(), consolidationApproach) + renamed := series.RenamedTo(newName) + renamed.SetConsolidationFunc(cf) + results = append(results, renamed) + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// offsetToZero offsets a metric or wildcard seriesList by subtracting the minimum +// value in the series from each data point. +func offsetToZero(ctx *common.Context, seriesList singlePathSpec) (ts.SeriesList, error) { + results := make([]*ts.Series, len(seriesList.Values)) + for idx, series := range seriesList.Values { + minimum := series.SafeMin() + numSteps := series.Len() + vals := ts.NewValues(ctx, series.MillisPerStep(), numSteps) + if !math.IsNaN(minimum) { + for i := 0; i < numSteps; i++ { + v := series.ValueAt(i) + if !math.IsNaN(v) { + vals.SetValueAt(i, v-minimum) + } + } + } + name := fmt.Sprintf("offsetToZero(%s)", series.Name()) + series := ts.NewSeries(ctx, name, series.StartTime(), vals) + results[idx] = series + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// timeFunction returns the timestamp for each X value. +// Note: step is measured in seconds. +func timeFunction(ctx *common.Context, name string, step int) (ts.SeriesList, error) { + if step <= 0 { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("step must be a positive int but instead is %d", step)) + } + + stepSizeInMilli := step * millisPerSecond + numSteps := ts.NumSteps(ctx.StartTime, ctx.EndTime, stepSizeInMilli) + vals := ts.NewValues(ctx, stepSizeInMilli, numSteps) + start := ctx.StartTime.Truncate(time.Second) + for current, index := start.Unix(), 0; index < numSteps; index++ { + vals.SetValueAt(index, float64(current)) + current += int64(step) + } + + series := ts.NewSeries(ctx, name, start, vals) + return ts.SeriesList{Values: []*ts.Series{series}}, nil +} + +// dashed draws the selected metrics with a dotted line with segments of length f. +func dashed(_ *common.Context, seriesList singlePathSpec, dashLength float64) (ts.SeriesList, error) { + if dashLength <= 0 { + return ts.SeriesList{}, errors.NewInvalidParamsError(fmt.Errorf("expected a positive dashLength but got %f", dashLength)) + } + + results := make([]*ts.Series, len(seriesList.Values)) + for idx, s := range seriesList.Values { + name := fmt.Sprintf("dashed(%s, "+common.FloatingPointFormat+")", s.Name(), dashLength) + renamed := s.RenamedTo(name) + results[idx] = renamed + } + + r := ts.SeriesList(seriesList) + r.Values = results + return r, nil +} + +// threshold draws a horizontal line at value f across the graph. +func threshold(ctx *common.Context, value float64, label string, color string) (ts.SeriesList, error) { + seriesList, err := constantLine(ctx, value) + if err != nil { + err := errors.NewInvalidParamsError(fmt.Errorf( + "error applying threshold function, error=%v", err)) + return ts.SeriesList{}, err + } + + series := seriesList.Values[0] + if label != "" { + series = series.RenamedTo(label) + } + + return ts.SeriesList{Values: []*ts.Series{series}}, nil +} + +func init() { + // functions - in alpha ordering + MustRegisterFunction(absolute) + MustRegisterFunction(aggregateLine).WithDefaultParams(map[uint8]interface{}{ + 2: "avg", // f + }) + MustRegisterFunction(alias) + MustRegisterFunction(aliasByMetric) + MustRegisterFunction(aliasByNode) + MustRegisterFunction(aliasSub) + MustRegisterFunction(asPercent).WithDefaultParams(map[uint8]interface{}{ + 2: []*ts.Series(nil), // total + }) + MustRegisterFunction(averageAbove) + MustRegisterFunction(averageSeries) + MustRegisterFunction(averageSeriesWithWildcards) + MustRegisterFunction(cactiStyle) + MustRegisterFunction(changed) + MustRegisterFunction(consolidateBy) + MustRegisterFunction(constantLine) + MustRegisterFunction(countSeries) + MustRegisterFunction(currentAbove) + MustRegisterFunction(currentBelow) + MustRegisterFunction(dashed).WithDefaultParams(map[uint8]interface{}{ + 2: 5.0, // dashLength + }) + MustRegisterFunction(derivative) + MustRegisterFunction(diffSeries) + MustRegisterFunction(divideSeries) + MustRegisterFunction(exclude) + MustRegisterFunction(fallbackSeries) + MustRegisterFunction(group) + MustRegisterFunction(groupByNode) + MustRegisterFunction(highestAverage) + MustRegisterFunction(highestCurrent) + MustRegisterFunction(highestMax) + MustRegisterFunction(hitcount) + MustRegisterFunction(holtWintersAberration) + MustRegisterFunction(holtWintersConfidenceBands) + MustRegisterFunction(holtWintersForecast) + MustRegisterFunction(identity) + MustRegisterFunction(integral) + MustRegisterFunction(isNonNull) + MustRegisterFunction(keepLastValue).WithDefaultParams(map[uint8]interface{}{ + 2: -1, // limit + }) + MustRegisterFunction(legendValue) + MustRegisterFunction(limit) + MustRegisterFunction(logarithm).WithDefaultParams(map[uint8]interface{}{ + 2: 10, // base + }) + MustRegisterFunction(lowestAverage) + MustRegisterFunction(lowestCurrent) + MustRegisterFunction(maxSeries) + MustRegisterFunction(maximumAbove) + MustRegisterFunction(minSeries) + MustRegisterFunction(minimumAbove) + MustRegisterFunction(mostDeviant) + MustRegisterFunction(movingAverage) + MustRegisterFunction(movingMedian) + MustRegisterFunction(multiplySeries) + MustRegisterFunction(nonNegativeDerivative).WithDefaultParams(map[uint8]interface{}{ + 2: math.NaN(), // maxValue + }) + MustRegisterFunction(nPercentile) + MustRegisterFunction(offset) + MustRegisterFunction(offsetToZero) + MustRegisterFunction(percentileOfSeries).WithDefaultParams(map[uint8]interface{}{ + 3: false, // interpolate + }) + MustRegisterFunction(perSecond).WithDefaultParams(map[uint8]interface{}{ + 2: math.NaN(), // maxValue + }) + MustRegisterFunction(rangeOfSeries) + MustRegisterFunction(randomWalkFunction).WithDefaultParams(map[uint8]interface{}{ + 2: 60, // step + }) + MustRegisterFunction(removeAbovePercentile) + MustRegisterFunction(removeAboveValue) + MustRegisterFunction(removeBelowPercentile) + MustRegisterFunction(removeBelowValue) + MustRegisterFunction(removeEmptySeries) + MustRegisterFunction(scale) + MustRegisterFunction(scaleToSeconds) + MustRegisterFunction(sortByMaxima) + MustRegisterFunction(sortByName) + MustRegisterFunction(sortByTotal) + MustRegisterFunction(squareRoot) + MustRegisterFunction(stdev).WithDefaultParams(map[uint8]interface{}{ + 3: 0.1, // windowTolerance + }) + MustRegisterFunction(substr).WithDefaultParams(map[uint8]interface{}{ + 2: 0, // start + 3: 0, // stop + }) + MustRegisterFunction(summarize).WithDefaultParams(map[uint8]interface{}{ + 3: "", // fname + 4: false, // alignToFrom + }) + MustRegisterFunction(sumSeries) + MustRegisterFunction(sumSeriesWithWildcards) + MustRegisterFunction(sustainedAbove) + MustRegisterFunction(sustainedBelow) + MustRegisterFunction(threshold).WithDefaultParams(map[uint8]interface{}{ + 2: "", // label + 3: "", // color + }) + MustRegisterFunction(timeFunction).WithDefaultParams(map[uint8]interface{}{ + 2: 60, // step + }) + MustRegisterFunction(timeShift).WithDefaultParams(map[uint8]interface{}{ + 3: true, // resetEnd + }) + MustRegisterFunction(transformNull).WithDefaultParams(map[uint8]interface{}{ + 2: 0.0, // defaultValue + }) + MustRegisterFunction(weightedAverage) + + // alias functions - in alpha ordering + MustRegisterAliasedFunction("abs", absolute) + MustRegisterAliasedFunction("avg", averageSeries) + MustRegisterAliasedFunction("log", logarithm) + MustRegisterAliasedFunction("max", maxSeries) + MustRegisterAliasedFunction("min", minSeries) + MustRegisterAliasedFunction("randomWalk", randomWalkFunction) + // NB(jayp): Graphite docs say that smartSummarize is the "smarter experimental version of + // summarize". Well, I am not sure about smarter, but aliasing satisfies the experimental + // aspect. + MustRegisterAliasedFunction("smartSummarize", summarize) + MustRegisterAliasedFunction("sum", sumSeries) + MustRegisterAliasedFunction("time", timeFunction) +} diff --git a/src/query/graphite/native/builtin_functions_test.go b/src/query/graphite/native/builtin_functions_test.go new file mode 100644 index 0000000000..fa7dfe693f --- /dev/null +++ b/src/query/graphite/native/builtin_functions_test.go @@ -0,0 +1,2868 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "math" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + xctx "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/storage" + xtest "github.com/m3db/m3/src/query/graphite/testing" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var ( + // testInput defines the input for various tests + testInput = []common.TestSeries{ + {"foo", []float64{0, 601, 3, 4}}, + {"nan", []float64{math.NaN(), math.NaN(), math.NaN()}}, + {"bar", []float64{500, -8}}, + {"baz", []float64{600, -600, 3}}, + {"quux", []float64{100, 50000, 888, -1, -2, math.NaN()}}, + } + + // testSmallInput defines a small input for various tests + testSmallInput = []common.TestSeries{ + testInput[0], + testInput[2], + } + + // testInputWithNaNSeries defines another input set with all-nan series + testInputWithNaNSeries = []common.TestSeries{ + testInput[0], + testInput[2], + testInput[4], + {"allNaN", []float64{math.NaN(), math.NaN()}}, + } +) + +func TestExclude(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + + g01 := ts.NewSeries(ctx, "servers.graphite01-foo.disk.bar.available_bytes", now, values) + g02 := ts.NewSeries(ctx, "servers.graphite02-foo.disk.bar.available_bytes", now, values) + g03 := ts.NewSeries(ctx, "servers.graphite03-foo.disk.bar.available_bytes", now, values) + + sampleInput := []*ts.Series{g01, g02, g03} + sampleOutput := []*ts.Series{g01, g03} + tests := []struct { + inputs []*ts.Series + r string + n int + outputs []*ts.Series + }{ + { + sampleInput, + "graphite02-foo", + 2, + sampleOutput, + }, + { + sampleInput, + "graphite", + 0, + []*ts.Series{}, + }, + { + sampleInput, + "graphite.*-foo", + 0, + []*ts.Series{}, + }, + } + + for _, test := range tests { + results, err := exclude(nil, singlePathSpec{ + Values: test.inputs, + }, test.r) + require.Nil(t, err) + require.NotNil(t, results) + require.Equal(t, test.n, results.Len()) + for i := range results.Values { + assert.Equal(t, sampleOutput[i].Name(), results.Values[i].Name()) + } + } +} + +func TestExcludeErr(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + + series := []*ts.Series{ + ts.NewSeries(ctx, "anything", now, values), + } + results, err := exclude(ctx, singlePathSpec{ + Values: series, + }, "(") + require.Error(t, err, "Failure is expected") + require.Nil(t, results.Values) +} + +func TestSortByName(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + values := ts.NewConstantValues(ctx, 10.0, 1000, 10) + + series := []*ts.Series{ + ts.NewSeries(ctx, "b.d.a", now, values), + ts.NewSeries(ctx, "zee", now, values), + ts.NewSeries(ctx, "a.c.d", now, values), + } + + results, err := sortByName(ctx, singlePathSpec{ + Values: series, + }) + require.Nil(t, err) + require.Equal(t, len(series), results.Len()) + assert.Equal(t, "a.c.d", results.Values[0].Name()) + assert.Equal(t, "b.d.a", results.Values[1].Name()) + assert.Equal(t, "zee", results.Values[2].Name()) +} + +func getTestInput(ctx *common.Context) []*ts.Series { + series := make([]*ts.Series, len(testInput)) + now := time.Now() + for idx, s := range testInput { + series[idx] = ts.NewSeries(ctx, s.Name, now, common.NewTestSeriesValues(ctx, 100, s.Data)) + } + return series +} + +func testSortingFuncs( + t *testing.T, + f func(ctx *common.Context, series singlePathSpec) (ts.SeriesList, error), + resultIndexes []int, +) { + ctx := common.NewTestContext() + defer ctx.Close() + + input := getTestInput(ctx) + results, err := f(ctx, singlePathSpec{Values: input}) + require.Nil(t, err) + require.Equal(t, len(resultIndexes), results.Len()) + for i, idx := range resultIndexes { + require.Equal(t, results.Values[i], input[idx]) + } +} + +func TestSortByTotal(t *testing.T) { + testSortingFuncs(t, sortByTotal, []int{4, 0, 2, 3, 1}) +} + +func TestSortByMaxima(t *testing.T) { + testSortingFuncs(t, sortByMaxima, []int{4, 0, 3, 2, 1}) +} + +func TestAbsolute(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + inputVals := []float64{-2, 0, 42, math.NaN()} + outputVals := []float64{2, 0, 42, math.NaN()} + start := time.Now() + + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 100, inputVals)) + r, err := absolute(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 100, outputs[0].MillisPerStep()) + require.Equal(t, len(outputVals), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + assert.Equal(t, "absolute(foo)", outputs[0].Name()) + + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + xtest.Equalish(t, outputVals[step], v, "invalid value for %d", step) + } +} + +func TestScale(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + scale float64 + outputs []float64 + }{ + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, + 2.5, + []float64{0, 2.5, 5.0, math.NaN(), 7.5}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, + 0.5, + []float64{0, 0.5, 1.0, math.NaN(), 1.5}, + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 100, test.inputs)) + r, err := scale(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, test.scale) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 100, outputs[0].MillisPerStep()) + require.Equal(t, len(test.inputs), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + assert.Equal(t, fmt.Sprintf("scale(foo,"+common.FloatingPointFormat+")", test.scale), outputs[0].Name()) + + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + xtest.Equalish(t, test.outputs[step], v, "invalid value for %d", step) + } + } +} + +func TestPercentileOfSeriesErrors(t *testing.T) { + ctx := common.NewTestContext() + + tests := []struct { + stepPerMillis []int + percentile float64 + values [][]float64 + expectedValues []float64 + expectedStepPerMillis float64 + interpolate genericInterface + }{ + { // percentile is over 100%. + []int{120, 120}, + 101.0, + [][]float64{ + {60.0, 50.0, 40.0, 30.0, 20.0, 10.0}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{5.5, 11.0, 16.5, 22.0, 27.5, 33.0}, + 120, + "true", + }, + { // percentile is less than zero. + []int{120, 120}, + -10.0, + [][]float64{ + {60.0, 50.0, 40.0, 30.0, 20.0, 10.0}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{5.5, 11.0, 16.5, 22.0, 27.5, 33.0}, + 120, + "true", + }, + { // percentile input is empty. + []int{120, 120}, + 10.0, + [][]float64{}, + []float64{}, + 120, + "true", + }, + { // percentile series have different size millisPerStep. + []int{120, 320}, + 33.0, + [][]float64{ + {60.0, 50.0, 40.0, 30.0, 20.0, 10.0}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{5.5, 11.0, 16.5, 22.0, 27.5, 33.0}, + 960, + "true", + }, + { // interpolateStr is neither "true" nor "false". + []int{120, 320}, + 33.0, + [][]float64{ + {60.0, 50.0, 40.0, 30.0, 20.0, 10.0}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{5.5, 11.0, 16.5, 22.0, 27.5, 33.0}, + 960, + "random", + }, + { // types other than boolean and string are not allowed + []int{120, 120, 120, 120, 120}, + 33.0, + [][]float64{ + {math.NaN(), 16, 23, math.NaN(), 75, 48, 42, 41}, + {math.NaN(), 36, 74, 43, 73}, + {math.NaN(), 61, 24, 29, math.NaN(), 62, 65, 72}, + {math.NaN(), 48, 94, math.NaN(), 32, 39, math.NaN(), 84}, + {math.NaN(), 16, math.NaN(), 85, 34, 27, 74, math.NaN(), 72}, + }, + []float64{math.NaN(), 16, 24, 43, 34}, + 120, + []*ts.Series(nil), + }, + } + + for _, test := range tests { + seriesList := make([]*ts.Series, len(test.values)) + for i := 0; i < len(seriesList); i++ { + seriesList[i] = ts.NewSeries(ctx, "", time.Now(), common.NewTestSeriesValues(ctx, test.stepPerMillis[i], test.values[i])) + } + + _, err := percentileOfSeries(ctx, singlePathSpec{ + Values: seriesList, + }, test.percentile, test.interpolate) + assert.NotNil(t, err) + } +} + +func TestPercentileOfSeries(t *testing.T) { + ctx := common.NewTestContext() + + tests := []struct { + stepPerMillis []int + percentile float64 + values [][]float64 + expectedValues []float64 + expectedStepPerMillis float64 + interpolate genericInterface + }{ + { // Test arrays with NaNs, multiple series, and same time step. + []int{120, 120, 120, 120, 120}, + 33, + [][]float64{ + {math.NaN(), 16, 23, math.NaN(), 75, 48, 42, 41}, + {math.NaN(), 36, 74, 43, 73}, + {math.NaN(), 61, 24, 29, math.NaN(), 62, 65, 72}, + {math.NaN(), 48, 94, math.NaN(), 32, 39, math.NaN(), 84}, + {math.NaN(), 16, math.NaN(), 85, 34, 27, 74, math.NaN(), 72}, + }, + []float64{math.NaN(), 16, 24, 29, 34}, + 120, + "false", + }, + { // Test arrays with NaNs, multiple series, and same time step. + []int{120, 120, 120, 120, 120}, + 33, + [][]float64{ + {math.NaN(), 16, 23, math.NaN(), 75, 48, 42, 41}, + {math.NaN(), 36, 74, 43, 73}, + {math.NaN(), 61, 24, 29, math.NaN(), 62, 65, 72}, + {math.NaN(), 48, 94, math.NaN(), 32, 39, math.NaN(), 84}, + {math.NaN(), 16, math.NaN(), 85, 34, 27, 74, math.NaN(), 72}, + }, + []float64{math.NaN(), 16.0, 23.32, 29, 32.64}, + 120, + "true", + }, + { // Test arrays with NaNs remove them and get correct percentile value + []int{120, 120, 120}, + 5, + [][]float64{ + {math.NaN(), 60, 50, 40, math.NaN(), 30, 20, 10}, + {math.NaN(), 15, 12, 9, 6, 3, math.NaN()}, + {math.NaN(), 6, 5, 4, 3, 2, 1}, + }, + []float64{math.NaN(), 6, 5, 4, 3, 2, 1}, + 120, + "false", + }, + { // Test non-interpolated percentile + []int{120, 120}, + 42, + [][]float64{ + {60, 5, 40, 30, 20, 10}, + {3, 40, 4, 1, 2, 6}, + }, + []float64{3, 5, 4, 1, 2, 6}, + 120, + "false", + }, + { // Test non-interpolated percentile for 100th percentile + []int{120, 120, 120}, + 100, + [][]float64{ + {60, 50, 40, 30, 20, 10}, + {18, 15, 12, 9, 6, 3}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{60, 50, 40, 30, 20, 10}, + 120, + "false", + }, + { // Test non-interpolated percentile for 1st percentile + []int{120, 120}, + 1, + [][]float64{ + {60, 50, 40, 30, 20, 10}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{6, 5, 4, 3, 2, 1}, + 120, + "false", + }, + { // Test interpolation for a percentile series + []int{120, 120}, + 75, + [][]float64{ + {60, 50, 40, 30, 20, 10}, + {6, 5, 4, 3, 2, 1}, + }, + []float64{33, 27.5, 22, 16.5, 11, 5.5}, + 120, + "true", + }, + } + + for _, test := range tests { + seriesList := make([]*ts.Series, len(test.values)) + for i := 0; i < len(seriesList); i++ { + seriesList[i] = ts.NewSeries(ctx, "", time.Now(), common.NewTestSeriesValues(ctx, test.stepPerMillis[i], test.values[i])) + } + + r, err := percentileOfSeries(ctx, singlePathSpec{ + Values: seriesList, + }, test.percentile, test.interpolate) + require.NoError(t, err) + + output := r.Values + name := fmt.Sprintf("percentileOfSeries(,"+common.FloatingPointFormat+")", + test.percentile) + assert.Equal(t, name, output[0].Name()) + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + require.NoError(t, err) + + xtest.Equalish(t, test.expectedValues[step], v) + } + xtest.Equalish(t, test.expectedStepPerMillis, output[0].MillisPerStep()) + } +} + +func TestOffset(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + factor float64 + outputs []float64 + }{ + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, 2.5, + []float64{2.5, 3.5, 4.5, math.NaN(), 5.5}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, -0.5, + []float64{-0.5, 0.5, 1.5, math.NaN(), 2.5}, + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 100, test.inputs)) + r, err := offset(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, test.factor) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 100, outputs[0].MillisPerStep()) + require.Equal(t, len(test.inputs), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + assert.Equal(t, fmt.Sprintf("offset(foo,"+common.FloatingPointFormat+")", test.factor), outputs[0].Name()) + + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + xtest.Equalish(t, test.outputs[step], v, "invalid value for %d", step) + } + } + +} + +func TestPerSecond(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + millisPerStep int + input []float64 + output []float64 + }{ + // increase by 1 per 100ms == 10 per sec + {100, []float64{1, 2, 3, 4, 5}, []float64{math.NaN(), 10, 10, 10, 10}}, + + // increase by 1 per 10s == .1 per sec + {10000, []float64{1, 2, 3, 4, 5}, []float64{math.NaN(), 0.1, 0.1, 0.1, 0.1}}, + + // decreasing value - rate of change not applicable + {1000, []float64{5, 4, 3, 2, 1}, + []float64{math.NaN(), math.NaN(), math.NaN(), math.NaN(), math.NaN()}}, + + // skip over missing values + {1000, []float64{1, 2, math.NaN(), 4, 5}, []float64{math.NaN(), 1, math.NaN(), 1, 1}}, + } + + for _, test := range tests { + values := common.NewTestSeriesValues(ctx, test.millisPerStep, test.input) + series := ts.NewSeries(ctx, "foo", time.Now(), values) + r, err := perSecond(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, math.NaN()) + require.NoError(t, err) + + perSec := r.Values + require.Equal(t, 1, len(perSec)) + require.Equal(t, len(test.output), perSec[0].Len()) + assert.Equal(t, series.StartTime(), perSec[0].StartTime()) + assert.Equal(t, "perSecond(foo)", perSec[0].Name()) + for i := 0; i < perSec[0].Len(); i++ { + val := perSec[0].ValueAt(i) + xtest.Equalish(t, test.output[i], val, "invalid value for %d", i) + } + } +} + +func TestTransformNull(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + defaultValue float64 + outputs []float64 + }{ + { + []float64{0, math.NaN(), 2.0, math.NaN(), 3.0}, 42.5, + []float64{0, 42.5, 2.0, 42.5, 3.0}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, -0.5, + []float64{0, 1.0, 2.0, -0.5, 3.0}, + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 100, test.inputs)) + r, err := transformNull(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, test.defaultValue) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 100, outputs[0].MillisPerStep()) + require.Equal(t, len(test.inputs), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + assert.Equal(t, fmt.Sprintf("transformNull(foo,"+common.FloatingPointFormat+")", test.defaultValue), outputs[0].Name()) + + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + assert.Equal(t, test.outputs[step], v, "invalid value for %d", step) + } + } +} + +var ( + testMovingAverageBootstrap = testMovingAverageStart.Add(-30 * time.Second) + testMovingAverageStart = time.Now().Truncate(time.Minute) + testMovingAverageEnd = testMovingAverageStart.Add(time.Minute) +) + +func testMovingAverage(t *testing.T, target, expectedName string, values, bootstrap, output []float64) { + ctx := common.NewTestContext() + defer ctx.Close() + + engine := NewEngine( + &common.MovingAverageStorage{ + StepMillis: 10000, + Bootstrap: bootstrap, + BootstrapStart: testMovingAverageBootstrap, + Values: values, + }, + ) + phonyContext := common.NewContext(common.ContextOptions{ + Start: testMovingAverageStart, + End: testMovingAverageEnd, + Engine: engine, + }) + + expr, err := phonyContext.Engine.(*Engine).Compile(target) + require.NoError(t, err) + res, err := expr.Execute(phonyContext) + require.NoError(t, err) + var expected []common.TestSeries + if output != nil { + expectedSeries := common.TestSeries{ + Name: expectedName, + Data: output, + } + expected = append(expected, expectedSeries) + } + common.CompareOutputsAndExpected(t, 10000, testMovingAverageStart, + expected, res.Values) +} + +func TestMovingAverageSuccess(t *testing.T) { + values := []float64{12.0, 19.0, -10.0, math.NaN(), 10.0} + bootstrap := []float64{3.0, 4.0, 5.0} + expected := []float64{4.0, 7.0, 12.0, 7.0, 4.5} + testMovingAverage(t, "movingAverage(foo.bar.baz, '30s')", "movingAverage(foo.bar.baz,\"30s\")", values, bootstrap, expected) + testMovingAverage(t, "movingAverage(foo.bar.baz, 3)", "movingAverage(foo.bar.baz,3)", values, bootstrap, expected) + testMovingAverage(t, "movingAverage(foo.bar.baz, 3)", "movingAverage(foo.bar.baz,3)", nil, nil, nil) + + bootstrapEntireSeries := []float64{3.0, 4.0, 5.0, 12.0, 19.0, -10.0, math.NaN(), 10.0} + testMovingAverage(t, "movingAverage(foo.bar.baz, '30s')", "movingAverage(foo.bar.baz,\"30s\")", values, bootstrapEntireSeries, expected) + testMovingAverage(t, "movingAverage(foo.bar.baz, 3)", "movingAverage(foo.bar.baz,3)", values, bootstrapEntireSeries, expected) +} + +func testMovingAverageError(t *testing.T, target string) { + ctx := common.NewTestContext() + defer ctx.Close() + + engine := NewEngine( + &common.MovingAverageStorage{ + StepMillis: 10000, + Bootstrap: []float64{1.0}, + BootstrapStart: testMovingAverageBootstrap, + Values: []float64{1.0}, + }, + ) + phonyContext := common.NewContext(common.ContextOptions{ + Start: testMovingAverageStart, + End: testMovingAverageEnd, + Engine: engine, + }) + + expr, err := phonyContext.Engine.(*Engine).Compile(target) + require.NoError(t, err) + res, err := expr.Execute(phonyContext) + require.Error(t, err) + require.Nil(t, res.Values) +} + +func TestMovingAverageError(t *testing.T) { + testMovingAverageError(t, "movingAverage(foo.bar.baz, '-30s')") + testMovingAverageError(t, "movingAverage(foo.bar.baz, 0)") +} + +func TestIsNonNull(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + outputs []float64 + }{ + { + []float64{0, math.NaN(), 2.0, math.NaN(), 3.0}, + []float64{1, 0, 1, 0, 1}, + }, + { + []float64{0, 1.0, 2.0, math.NaN(), 3.0}, + []float64{1, 1, 1, 0, 1}, + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 100, test.inputs)) + r, err := isNonNull(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 100, outputs[0].MillisPerStep()) + require.Equal(t, len(test.inputs), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + assert.Equal(t, "isNonNull(foo)", outputs[0].Name()) + + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + assert.Equal(t, test.outputs[step], v, "invalid value for %d", step) + } + } +} + +func TestKeepLastValue(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + outputs []float64 + limit int + }{ + { + []float64{0, math.NaN(), 2.0, math.NaN(), 3.0}, + []float64{0, 0, 2.0, 2.0, 3.0}, + -1, + }, + { + []float64{math.NaN(), 1.0, 2.0, math.NaN(), 3.0}, + []float64{math.NaN(), 1.0, 2.0, 2.0, 3.0}, + -1, + }, + { + []float64{1.0, math.NaN(), math.NaN(), math.NaN(), 3.0, math.NaN(), math.NaN(), 2.0}, + []float64{1.0, math.NaN(), math.NaN(), math.NaN(), 3.0, 3.0, 3.0, 2.0}, + 2, + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 100, test.inputs)) + outputs, err := keepLastValue(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, test.limit) + expected := common.TestSeries{Name: "keepLastValue(foo)", Data: test.outputs} + require.NoError(t, err) + common.CompareOutputsAndExpected(t, 100, start, + []common.TestSeries{expected}, outputs.Values) + } +} + +func TestSustainedAbove(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + outputs []float64 + threshold float64 + interval string + }{ + { + []float64{0, 0, 3, 3, 4, 0, 0}, + []float64{0, 0, 3, 3, 4, 0, 0}, + 2, + "10s", + }, + { + []float64{0, 0, 3, 3, 4, 0, 0}, + []float64{0, 0, 0, 3, 4, 0, 0}, + 2, + "20s", + }, + { + []float64{0, 0, 3, 3, 4, 0, 0}, + []float64{0, 0, 0, 0, 4, 0, 0}, + 2, + "30s", + }, + { + []float64{0, 0, 3, 3, 4, 0, 0}, + []float64{0, 0, 0, 0, 0, 0, 0}, + 2, + "40s", + }, + { + []float64{0, 3, 3, 4, 4, 2, 0}, + []float64{0, 0, 0, 0, 4, 0, 0}, + 4, + "20s", + }, + { + []float64{1, 2, 3, 4, 9, 9, 9, 9, 9, 3}, + []float64{0, 0, 0, 0, 0, 0, 9, 9, 9, 0}, + 8, + "30s", + }, + { + []float64{1, 2, 3, 4, 5, 5, 5, 5, 5, 3}, + []float64{0, 0, 0, 4, 5, 5, 5, 5, 5, 0}, + 4, + "10s", + }, + { + []float64{-3, -4, -1, 3, 0, -1, -5, -6, -3}, + []float64{-4, -4, -4, 3, 0, -1, -4, -4, -4}, + -2, + "20s", + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 10000, test.inputs)) + r, err := sustainedAbove(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, test.threshold, test.interval) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 10000, outputs[0].MillisPerStep()) + require.Equal(t, len(test.inputs), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + + str := fmt.Sprintf("sustainedAbove(foo, %f, '%s')", test.threshold, test.interval) + + assert.Equal(t, str, outputs[0].Name()) + + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + + assert.Equal(t, test.outputs[step], v, "invalid value for %d", step) + } + } +} + +func TestSustainedAboveFail(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + input := ts.NewSeries(ctx, "foo", time.Now(), common.NewTestSeriesValues(ctx, 10000, []float64{0})) + outputs, err := sustainedAbove(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, 10, "wat") + require.Error(t, err) + require.Equal(t, 0, outputs.Len()) +} + +func TestSustainedBelow(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + inputs []float64 + outputs []float64 + threshold float64 + interval string + }{ + { + []float64{4, 4, 1, 1, 1, 4, 4}, + []float64{4, 4, 1, 1, 1, 4, 4}, + 2, + "10s", + }, + { + []float64{7, 8, 3, 3, 2, 6, 7}, + []float64{6, 6, 6, 3, 2, 6, 6}, + 3, + "20s", + }, + { + []float64{9, 7, 3, 3, 2, 5, 6}, + []float64{6, 6, 6, 6, 2, 6, 6}, + 3, + "30s", + }, + { + []float64{8, 5, 3, 3, 2, 5, 8}, + []float64{6, 6, 6, 6, 6, 6, 6}, + 3, + "40s", + }, + { + []float64{4, 3, 3, 1, 1, 2, 4}, + []float64{2, 2, 2, 2, 1, 2, 2}, + 1, + "20s", + }, + { + []float64{7, 8, 9, 2, 2, 4, 2, 5, 3, 2}, + []float64{8, 8, 8, 8, 8, 8, 2, 8, 8, 8}, + 4, + "40s", + }, + { + []float64{1, 2, 3, 4, 9, 9, 9, 9, 9, 3}, + []float64{8, 2, 3, 4, 8, 8, 8, 8, 8, 8}, + 4, + "20s", + }, + { + []float64{-3, -4, -3, -1, 3, 2, -5, -4, -3, -3}, + []float64{0, -4, -3, 0, 0, 0, 0, -4, -3, -3}, + -2, + "20s", + }, + } + + start := time.Now() + for _, test := range tests { + input := ts.NewSeries(ctx, "foo", start, common.NewTestSeriesValues(ctx, 10000, test.inputs)) + r, err := sustainedBelow(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, test.threshold, test.interval) + require.NoError(t, err) + + outputs := r.Values + require.Equal(t, 1, len(outputs)) + require.Equal(t, 10000, outputs[0].MillisPerStep()) + require.Equal(t, len(test.inputs), outputs[0].Len()) + require.Equal(t, start, outputs[0].StartTime()) + + str := fmt.Sprintf("sustainedBelow(foo, %f, '%s')", test.threshold, test.interval) + + assert.Equal(t, str, outputs[0].Name()) + for step := 0; step < outputs[0].Len(); step++ { + v := outputs[0].ValueAt(step) + + assert.Equal(t, test.outputs[step], v, "invalid value for %d", step) + } + } +} + +func TestSustainedBelowFail(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + input := ts.NewSeries(ctx, "foo", time.Now(), common.NewTestSeriesValues(ctx, 10000, []float64{0})) + outputs, err := sustainedBelow(ctx, singlePathSpec{ + Values: []*ts.Series{input}, + }, 10, "wat") + require.Error(t, err) + require.Equal(t, 0, outputs.Len()) +} + +// nIntParamGoldenData holds test data for functions that take an additional "n" int parameter +type nIntParamGoldenData struct { + inputs []common.TestSeries + n int + outputs []common.TestSeries +} + +// rankingFunc selects the n lowest or highest series based on certain metric of the +// series (e.g., maximum, minimum, average). +type rankingFunc func(ctx *common.Context, input singlePathSpec, n int) (ts.SeriesList, error) + +func testRanking(t *testing.T, ctx *common.Context, tests []nIntParamGoldenData, f rankingFunc) { + start := time.Now() + step := 100 + for _, test := range tests { + outputs, err := f(ctx, singlePathSpec{ + Values: generateSeriesList(ctx, start, test.inputs, step), + }, test.n) + if test.n < 0 { + require.NotNil(t, err) + require.Equal(t, "n must be positive", err.Error()) + assert.Nil(t, outputs.Values, "Nil timeseries should be returned") + continue + } + require.NoError(t, err) + common.CompareOutputsAndExpected(t, step, start, + test.outputs, outputs.Values) + } +} + +func TestHighestCurrent(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInput, + 0, + nil, + }, + { + testInput, + 1, + []common.TestSeries{testInput[0]}, + }, + { + testInput, + 2, + []common.TestSeries{testInput[0], testInput[3]}, + }, + { + testInput, + len(testInput) + 10, // force sort + []common.TestSeries{testInput[0], testInput[3], testInput[4], testInput[2], testInput[1]}, + }, + } + testRanking(t, ctx, tests, highestCurrent) +} + +func TestHighestCurrentWithNaNSeries(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInputWithNaNSeries, + 0, + nil, + }, + { + testInputWithNaNSeries, + 1, + []common.TestSeries{testInputWithNaNSeries[0]}, + }, + { + testInputWithNaNSeries, + 2, + []common.TestSeries{testInputWithNaNSeries[0], testInputWithNaNSeries[2]}, + }, + { + testInputWithNaNSeries, + 3, + []common.TestSeries{testInputWithNaNSeries[0], testInputWithNaNSeries[2], testInputWithNaNSeries[1]}, + }, + { + testInputWithNaNSeries, + 4, + []common.TestSeries{testInputWithNaNSeries[0], testInputWithNaNSeries[2], testInputWithNaNSeries[1], testInputWithNaNSeries[3]}, + }, + } + testRanking(t, ctx, tests, highestCurrent) +} + +func TestHighestAverage(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInput, + 1, + []common.TestSeries{testInput[4]}, + }, + { + testInput, + 2, + []common.TestSeries{testInput[4], testInput[2]}, + }, + } + testRanking(t, ctx, tests, highestAverage) +} + +func TestHighestMax(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInput, + 1, + []common.TestSeries{testInput[4]}, + }, + { + testInput, + 2, + []common.TestSeries{testInput[4], testInput[0]}, + }, + } + testRanking(t, ctx, tests, highestMax) +} + +func TestFallbackSeries(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + input []common.TestSeries + fallback []common.TestSeries + output []common.TestSeries + }{ + { + nil, + []common.TestSeries{common.TestSeries{"output", []float64{0, 1.0}}}, + []common.TestSeries{common.TestSeries{"output", []float64{0, 1.0}}}, + }, + { + []common.TestSeries{}, + []common.TestSeries{common.TestSeries{"output", []float64{0, 1.0}}}, + []common.TestSeries{common.TestSeries{"output", []float64{0, 1.0}}}, + }, + { + []common.TestSeries{common.TestSeries{"output", []float64{0, 2.0}}}, + []common.TestSeries{common.TestSeries{"fallback", []float64{0, 1.0}}}, + []common.TestSeries{common.TestSeries{"output", []float64{0, 2.0}}}, + }, + } + + start := time.Now() + step := 100 + for _, test := range tests { + + inputs := generateSeriesList(ctx, start, test.input, step) + fallbacks := generateSeriesList(ctx, start, test.fallback, step) + + outputs, err := fallbackSeries(ctx, singlePathSpec{ + Values: inputs, + }, singlePathSpec{ + Values: fallbacks, + }) + require.NoError(t, err) + + common.CompareOutputsAndExpected(t, step, start, + test.output, outputs.Values) + } +} + +func TestMostDeviant(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInput, + -2, + nil, + }, + { + testInput, + 1, + []common.TestSeries{testInput[4]}, + }, + { + testInput, + 2, + []common.TestSeries{testInput[4], testInput[3]}, + }, + } + testRanking(t, ctx, tests, mostDeviant) +} + +func TestLowestAverage(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInput, + 0, + nil, + }, + { + testInput, + 1, + []common.TestSeries{testInput[1]}, + }, + { + testInput, + 2, + []common.TestSeries{testInput[1], testInput[3]}, + }, + { + testInput, + 3, + []common.TestSeries{testInput[1], testInput[3], testInput[0]}, + }, + } + testRanking(t, ctx, tests, lowestAverage) +} + +func TestLowestCurrent(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []nIntParamGoldenData{ + { + testInput, + 0, + nil, + }, + { + testInput, + 1, + []common.TestSeries{testInput[1]}, + }, + { + testInput, + 2, + []common.TestSeries{testInput[1], testInput[2]}, + }, + { + testInput, + 3, + []common.TestSeries{testInput[1], testInput[2], testInput[4]}, + }, + } + testRanking(t, ctx, tests, lowestCurrent) +} + +type comparatorFunc func(ctx *common.Context, series singlePathSpec, n float64) (ts.SeriesList, error) + +func testComparatorFunc( + t *testing.T, + f comparatorFunc, + n float64, + resultIndexes []int, +) { + ctx := common.NewTestContext() + defer ctx.Close() + + input := getTestInput(ctx) + results, err := f(ctx, singlePathSpec{ + Values: input, + }, n) + require.Nil(t, err) + require.Equal(t, len(resultIndexes), results.Len()) + for i, idx := range resultIndexes { + require.Equal(t, input[idx], results.Values[i]) + } +} + +func TestMaximumAbove(t *testing.T) { + testComparatorFunc(t, maximumAbove, -10, []int{0, 2, 3, 4}) + testComparatorFunc(t, maximumAbove, 600, []int{0, 4}) + testComparatorFunc(t, maximumAbove, 100000, nil) +} + +func TestMinimumAbove(t *testing.T) { + testComparatorFunc(t, minimumAbove, -1000, []int{0, 2, 3, 4}) + testComparatorFunc(t, minimumAbove, -100, []int{0, 2, 4}) + testComparatorFunc(t, minimumAbove, 1, nil) +} + +func TestAverageAbove(t *testing.T) { + testComparatorFunc(t, averageAbove, 0, []int{0, 2, 3, 4}) + testComparatorFunc(t, averageAbove, 1, []int{0, 2, 4}) + testComparatorFunc(t, averageAbove, 12000, nil) +} + +func TestCurrentAbove(t *testing.T) { + testComparatorFunc(t, currentAbove, -10, []int{0, 2, 3, 4}) + testComparatorFunc(t, currentAbove, -5, []int{0, 3, 4}) + testComparatorFunc(t, currentAbove, 5, nil) +} + +func TestCurrentBelow(t *testing.T) { + testComparatorFunc(t, currentBelow, 5, []int{0, 2, 3, 4}) + testComparatorFunc(t, currentBelow, 0, []int{2, 4}) + testComparatorFunc(t, currentBelow, -5, []int{2}) + testComparatorFunc(t, currentBelow, -10, nil) +} + +func TestRemoveBelowValue(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + tests := []struct { + inputs []common.TestSeries + n float64 + outputs []common.TestSeries + }{ + { + testSmallInput, + 500, + []common.TestSeries{ + {"foo", []float64{nan, 601, nan, nan}}, + {"bar", []float64{500, nan}}, + }, + }, + { + testSmallInput, + 4, + []common.TestSeries{ + {"foo", []float64{nan, 601, nan, 4}}, + {"bar", []float64{500, nan}}, + }, + }, + } + start := time.Now() + step := 100 + for _, test := range tests { + outputs, err := removeBelowValue(ctx, singlePathSpec{ + Values: generateSeriesList(ctx, start, test.inputs, step), + }, test.n) + require.NoError(t, err) + for i := range test.outputs { // overwrite series names + name := fmt.Sprintf("removeBelowValue(%s, "+common.FloatingPointFormat+")", + test.outputs[i].Name, test.n) + test.outputs[i].Name = name + } + common.CompareOutputsAndExpected(t, step, start, + test.outputs, outputs.Values) + } +} + +func TestRemoveAboveValue(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + tests := []struct { + inputs []common.TestSeries + n float64 + outputs []common.TestSeries + }{ + { + testSmallInput, + 500, + []common.TestSeries{ + {"foo", []float64{0, nan, 3, 4}}, + {"bar", []float64{500, -8}}, + }, + }, + { + testSmallInput, + 3, + []common.TestSeries{ + {"foo", []float64{0, nan, 3, nan}}, + {"bar", []float64{nan, -8}}, + }, + }, + } + start := time.Now() + step := 100 + for _, test := range tests { + outputs, err := removeAboveValue(ctx, singlePathSpec{ + Values: generateSeriesList(ctx, start, test.inputs, step), + }, test.n) + require.NoError(t, err) + for i := range test.outputs { // overwrite series names + test.outputs[i].Name = fmt.Sprintf( + "removeAboveValue(%s, "+common.FloatingPointFormat+")", + test.outputs[i].Name, + test.n, + ) + } + common.CompareOutputsAndExpected(t, step, start, + test.outputs, outputs.Values) + } +} + +func TestRemoveEmptySeries(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + tests := []struct { + inputs []common.TestSeries + outputs []common.TestSeries + }{ + { + []common.TestSeries{ + {"foo", []float64{nan, 601, nan, nan}}, + {"bar", []float64{500, nan}}, + {"baz", []float64{nan, nan, nan}}, + }, + []common.TestSeries{ + {"foo", []float64{nan, 601, nan, nan}}, + {"bar", []float64{500, nan}}, + }, + }, + } + start := time.Now() + step := 100 + for _, test := range tests { + outputs, err := removeEmptySeries(ctx, singlePathSpec{ + Values: generateSeriesList(ctx, start, test.inputs, step), + }) + require.NoError(t, err) + common.CompareOutputsAndExpected(t, step, start, + test.outputs, outputs.Values) + } +} + +func generateSeriesList(ctx *common.Context, start time.Time, inputs []common.TestSeries, step int) []*ts.Series { + tSeriesList := make([]*ts.Series, 0, len(inputs)) + for _, in := range inputs { + tSeries := ts.NewSeries(ctx, in.Name, start, common.NewTestSeriesValues(ctx, step, in.Data)) + tSeriesList = append(tSeriesList, tSeries) + } + return tSeriesList +} + +func TestScaleToSeconds(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + millisPerStep int + values []float64 + expected []float64 + seconds int + }{ + { + 1000, + []float64{1000.0, 2000.0, 3000.0, 4000.0, 5000.0}, + []float64{2000.0, 4000.0, 6000.0, 8000.0, 10000.0}, + 2, + }, + // expected values should double when step is halved + // relative to the original expected values + { + 500, + []float64{1000.0, 2000.0, 3000.0, 4000.0, 5000.0}, + []float64{4000.0, 8000.0, 12000.0, 16000.0, 20000.0}, + 2, + }, + // expected values should drop by a factor of 1/5 when step is multiplied by 5 + // relative to the original expected values + { + 5000, + []float64{1000.0, 2000.0, 3000.0, 4000.0, 5000.0}, + []float64{400.0, 800.0, 1200.0, 1600.0, 2000.0}, + 2, + }, + } + + for _, test := range tests { + timeSeries := ts.NewSeries(ctx, "", ctx.StartTime, + common.NewTestSeriesValues(ctx, test.millisPerStep, test.values)) + + r, err := scaleToSeconds(ctx, singlePathSpec{ + Values: []*ts.Series{timeSeries}, + }, test.seconds) + require.NoError(t, err) + + output := r.Values + require.Equal(t, 1, len(output)) + assert.Equal(t, "scaleToSeconds(,2)", output[0].Name()) + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + assert.Equal(t, test.expected[step], v) + } + } +} + +func TestAsPercentWithSeriesTotal(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + valuesStep int + values []float64 + totalsStep int + totals []float64 + outputStep int + output []float64 + }{ + { + 100, []float64{10.0, 20.0, 30.0, 40.0, 50.0}, + 100, []float64{1000.0, 1000.0, 1000.0, 1000.0, 1000.0}, + 100, []float64{1.0, 2.0, 3.0, 4.0, 5.0}, + }, + { + 100, []float64{12.0, 14.0, 16.0, math.NaN(), 20.0}, + 150, []float64{50.0, 50.0, 25.0, 50.0, 50.0}, + 300, []float64{28.0, 53.0}, + }, + } + + for _, test := range tests { + timeSeries := ts.NewSeries(ctx, "", ctx.StartTime, + common.NewTestSeriesValues(ctx, test.valuesStep, test.values)) + totalSeries := ts.NewSeries(ctx, "", ctx.StartTime, + common.NewTestSeriesValues(ctx, test.totalsStep, test.totals)) + + r, err := asPercent(ctx, singlePathSpec{ + Values: []*ts.Series{timeSeries}, + }, ts.SeriesList{ + Values: []*ts.Series{totalSeries}, + }) + require.NoError(t, err, fmt.Sprintf("err: %v", err)) + + output := r.Values + require.Equal(t, 1, len(output)) + require.Equal(t, output[0].MillisPerStep(), test.outputStep) + assert.Equal(t, "asPercent(, )", output[0].Name()) + + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + assert.Equal(t, math.Trunc(v), test.output[step]) + } + } +} + +func TestAsPercentWithFloatTotal(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + tests := []struct { + valuesStep int + values []float64 + total float64 + outputStep int + output []float64 + }{ + { + 100, []float64{12.0, 14.0, 16.0, nan, 20.0}, + 20.0, + 100, []float64{60, 70, 80, nan, 100}, + }, + { + 100, []float64{12.0, 14.0, 16.0, nan, 20.0}, + 0, + 100, []float64{nan, nan, nan, nan, nan}, + }, + } + + for _, test := range tests { + timeSeries := ts.NewSeries(ctx, "", ctx.StartTime, + common.NewTestSeriesValues(ctx, test.valuesStep, test.values)) + r, err := asPercent(ctx, singlePathSpec{ + Values: []*ts.Series{timeSeries}, + }, test.total) + require.NoError(t, err) + + output := r.Values + require.Equal(t, 1, len(output)) + require.Equal(t, output[0].MillisPerStep(), test.outputStep) + expectedName := fmt.Sprintf("asPercent(, "+common.FloatingPointFormat+")", + test.total) + assert.Equal(t, expectedName, output[0].Name()) + + for step := 0; step < output[0].Len(); step++ { + v := output[0].ValueAt(step) + xtest.Equalish(t, math.Trunc(v), test.output[step]) + } + } +} + +func TestAsPercentWithSeriesList(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + inputs := []struct { + name string + step int + values []float64 + }{ + { + "foo", + 100, + []float64{12.0, 14.0, 16.0, nan, 20.0, 30.0}, + }, + { + "bar", + 200, + []float64{7.0, nan, 25.0}, + }, + } + outputs := []struct { + name string + step int + values []float64 + }{ + { + "asPercent(foo, foo)", + 200, + []float64{65.0, 100.0, 50.0}, + }, + { + "asPercent(bar, bar)", + 200, + []float64{35.0, nan, 50.0}, + }, + } + + var inputSeries []*ts.Series + for _, input := range inputs { + timeSeries := ts.NewSeries( + ctx, + input.name, + ctx.StartTime, + common.NewTestSeriesValues(ctx, input.step, input.values), + ) + inputSeries = append(inputSeries, timeSeries) + } + + var expected []*ts.Series + for _, output := range outputs { + timeSeries := ts.NewSeries( + ctx, + output.name, + ctx.StartTime, + common.NewTestSeriesValues(ctx, output.step, output.values), + ) + expected = append(expected, timeSeries) + } + + for _, totalArg := range []interface{}{ + ts.SeriesList{Values: []*ts.Series(nil)}, + singlePathSpec{}, + } { + r, err := asPercent(ctx, singlePathSpec{ + Values: inputSeries, + }, totalArg) + require.NoError(t, err) + + results := r.Values + require.Equal(t, len(expected), len(results)) + for i := 0; i < len(results); i++ { + require.Equal(t, expected[i].MillisPerStep(), results[i].MillisPerStep()) + require.Equal(t, expected[i].Len(), results[i].Len()) + require.Equal(t, expected[i].Name(), results[i].Name()) + for step := 0; step < results[i].Len(); step++ { + xtest.Equalish(t, expected[i].ValueAt(step), results[i].ValueAt(step)) + } + } + } + +} + +func testLogarithm(t *testing.T, base int, indices []int) { + ctx := common.NewTestContext() + defer ctx.Close() + + invals := make([]float64, 101) + for i := range invals { + invals[i] = float64(i) + } + + series := ts.NewSeries(ctx, "hello", time.Now(), + common.NewTestSeriesValues(ctx, 10000, invals)) + + r, err := logarithm(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, base) + require.NoError(t, err) + + output := r.Values + require.Equal(t, 1, len(output)) + assert.Equal(t, fmt.Sprintf("log(hello, %d)", base), output[0].Name()) + assert.Equal(t, series.StartTime(), output[0].StartTime()) + require.Equal(t, len(invals), output[0].Len()) + xtest.Equalish(t, math.NaN(), output[0].ValueAt(0)) + xtest.Equalish(t, 0, output[0].ValueAt(indices[0])) + xtest.Equalish(t, 1, output[0].ValueAt(indices[1])) + xtest.Equalish(t, 2, output[0].ValueAt(indices[2])) +} + +func TestLogarithm(t *testing.T) { + testLogarithm(t, 10, []int{1, 10, 100}) + testLogarithm(t, 2, []int{1, 2, 4}) + + _, err := logarithm(nil, singlePathSpec{}, -1) + require.NotNil(t, err) +} + +func TestIntegral(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + invals := []float64{ + 0, 1, 2, 3, 4, 5, 6, math.NaN(), 8, math.NaN(), + } + + outvals := []float64{ + 0, 1, 3, 6, 10, 15, 21, math.NaN(), 29, math.NaN(), + } + + series := ts.NewSeries(ctx, "hello", time.Now(), + common.NewTestSeriesValues(ctx, 10000, invals)) + + r, err := integral(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }) + require.NoError(t, err) + + output := r.Values + require.Equal(t, 1, len(output)) + assert.Equal(t, "integral(hello)", output[0].Name()) + assert.Equal(t, series.StartTime(), output[0].StartTime()) + require.Equal(t, len(outvals), output[0].Len()) + for i, expected := range outvals { + xtest.Equalish(t, expected, output[0].ValueAt(i), "incorrect value at %d", i) + } +} + +func TestDerivative(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + values []float64 + output []float64 + }{ + { + []float64{10.0, 20.0, 30.0, 5.0, 5.0}, + []float64{math.NaN(), 10.0, 10.0, -25.0, 0.0}, + }, + { + []float64{50.0, 50.0, 25.0, 250.0, 350.0}, + []float64{math.NaN(), 0.0, -25.0, 225.0, 100.0}, + }, + } + + start := time.Now() + step := 100 + for _, test := range tests { + input := []common.TestSeries{{"foo", test.values}} + expected := []common.TestSeries{{"derivative(foo)", test.output}} + timeSeries := generateSeriesList(ctx, start, input, step) + output, err := derivative(ctx, singlePathSpec{ + Values: timeSeries, + }) + require.NoError(t, err) + common.CompareOutputsAndExpected(t, step, start, + expected, output.Values) + } +} + +func TestNonNegativeDerivative(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + tests := []struct { + values []float64 + maxValue float64 + output []float64 + }{ + { + []float64{10.0, 20.0, 30.0, 5.0, 5.0}, + math.NaN(), + []float64{math.NaN(), 10.0, 10.0, math.NaN(), 0.0}, + }, + { + []float64{50.0, 50.0, 25.0, 250.0, 350.0}, + 100.0, + []float64{math.NaN(), 0.0, 76.0, 225.0, 100.0}, + }, + } + + start := time.Now() + step := 100 + for _, test := range tests { + input := []common.TestSeries{{"foo", test.values}} + expected := []common.TestSeries{{"nonNegativeDerivative(foo)", test.output}} + timeSeries := generateSeriesList(ctx, start, input, step) + output, err := nonNegativeDerivative(ctx, singlePathSpec{ + Values: timeSeries, + }, test.maxValue) + require.NoError(t, err) + common.CompareOutputsAndExpected(t, step, start, expected, output.Values) + } +} + +type TimeSeriesPtrVector []*ts.Series + +func (o TimeSeriesPtrVector) Len() int { return len(o) } +func (o TimeSeriesPtrVector) Less(i, j int) bool { return o[i].Name() < o[j].Name() } +func (o TimeSeriesPtrVector) Swap(i, j int) { o[i], o[j] = o[j], o[i] } + +func TestConstantLine(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + testValue := 5.0 + r, err := constantLine(ctx, testValue) + require.Nil(t, err) + + testSeries := r.Values + require.Equal(t, 1, len(testSeries)) + require.Equal(t, 2, testSeries[0].Len()) + expectedName := fmt.Sprintf(common.FloatingPointFormat, testValue) + require.Equal(t, expectedName, testSeries[0].Name()) + for i := 0; i < testSeries[0].Len(); i++ { + require.Equal(t, float64(testValue), testSeries[0].ValueAt(i)) + } +} + +func TestIdentity(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + testName := "testName.mytest" + r, err := identity(ctx, testName) + require.Nil(t, err) + + testSeries := r.Values + require.Equal(t, 1, len(testSeries)) + require.Equal(t, testName, testSeries[0].Name()) + require.Equal(t, 60, testSeries[0].Len()) + expectedValue := ctx.StartTime.Unix() + for i := 0; i < testSeries[0].Len(); i++ { + require.Equal(t, float64(expectedValue), testSeries[0].ValueAt(i)) + expectedValue += 60 + } +} + +func TestLimit(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + // invalid input + testInput := getTestInput(ctx) + testSeries, err := limit(ctx, singlePathSpec{ + Values: testInput, + }, -1) + require.NotNil(t, err) + + // valid input + testSeries, err = limit(ctx, singlePathSpec{ + Values: testInput, + }, 1) + require.Nil(t, err) + require.Equal(t, 1, testSeries.Len()) + + // input bigger than length of series + testSeries, err = limit(ctx, singlePathSpec{ + Values: testInput, + }, 10) + require.Nil(t, err) + require.Equal(t, len(testInput), testSeries.Len()) +} + +func TestHitCount(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := time.Now() + tests := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + intervalString string + newStartTime time.Time + newStep int + output []float64 + }{ + { + "foo", + now, + 1000, + []float64{1.0, 2.0, 3.0, 4.0, 5.0, math.NaN(), 6.0}, + "2s", + now.Add(-time.Second), + 2000, + []float64{1.0, 5.0, 9.0, 6.0}, + }, + { + "bar", + now, + 1000, + []float64{1.0, 2.0, 3.0, 4.0, 5.0, math.NaN(), 6.0}, + "10s", + now.Add(-3 * time.Second), + 10000, + []float64{21.0}, + }, + } + + for _, input := range tests { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + results, err := hitcount(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, input.intervalString) + expected := common.TestSeries{ + Name: fmt.Sprintf(`hitcount(%s, %q)`, input.name, input.intervalString), + Data: input.output, + } + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.newStep, input.newStartTime, + []common.TestSeries{expected}, results.Values) + } +} + +func TestSubstr(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + now := ctx.StartTime + input := struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + "aliasByName(foo.bar,baz)", + now, + 1000, + []float64{1.0, 2.0, 3.0}, + } + + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + results, err := substr(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 1, 0) + expected := common.TestSeries{Name: "bar", Data: input.values} + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.stepInMilli, input.startTime, + []common.TestSeries{expected}, results.Values) + + results, err = substr(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 0, 2) + expected = common.TestSeries{Name: "foo.bar", Data: input.values} + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.stepInMilli, input.startTime, + []common.TestSeries{expected}, results.Values) + + results, err = substr(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 0, 0) + expected = common.TestSeries{Name: "foo.bar", Data: input.values} + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.stepInMilli, input.startTime, + []common.TestSeries{expected}, results.Values) + + results, err = substr(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 2, 1) + require.NotNil(t, err) + + results, err = substr(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, -1, 1) + require.NotNil(t, err) + + results, err = substr(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 3, 4) + require.NotNil(t, err) +} + +type mockStorage struct{} + +func (*mockStorage) FetchByQuery( + ctx xctx.Context, query string, opts storage.FetchOptions, +) (*storage.FetchResult, error) { + return storage.NewFetchResult(ctx, nil), nil +} + +func TestHoltWintersForecast(t *testing.T) { + ctx := common.NewTestContext() + ctx.Engine = NewEngine( + &mockStorage{}, + ) + defer ctx.Close() + + now := ctx.StartTime + tests := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + duration time.Duration + newStartTime time.Time + newStep int + output []float64 + }{ + { + "foo", + now, + 1000, + []float64{4.0, 5.0, 6.0}, + 3 * time.Second, + now, + 1000, + []float64{4.0, 4.0, 4.10035}, + }, + } + + for _, input := range tests { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + results, err := holtWintersForecastInternal(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, input.duration) + expected := common.TestSeries{ + Name: fmt.Sprintf(`holtWintersForecast(%s)`, input.name), + Data: input.output, + } + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.newStep, input.newStartTime, + []common.TestSeries{expected}, results.Values) + } +} + +func TestHoltWintersConfidenceBands(t *testing.T) { + ctx := common.NewTestContext() + ctx.Engine = NewEngine( + &mockStorage{}, + ) + defer ctx.Close() + + now := ctx.StartTime + tests := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + duration time.Duration + lowerStartTime time.Time + lowerStep int + lowerOutput []float64 + upperStartTime time.Time + upperStep int + upperOutput []float64 + }{ + { + "foo", + now, + 1000, + []float64{4.0, 5.0, 6.0}, + 3 * time.Second, + now, + 1000, + []float64{0.4787, 3.7, 3.5305}, + now, + 1000, + []float64{2.1039, 4.3, 4.6702}, + }, + } + + for _, input := range tests { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + results, err := holtWintersConfidenceBandsInternal(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 3, input.duration) + lowerExpected := common.TestSeries{ + Name: fmt.Sprintf(`holtWintersConfidenceLower(%s)`, input.name), + Data: input.lowerOutput, + } + upperExpected := common.TestSeries{ + Name: fmt.Sprintf(`holtWintersConfidenceUpper(%s)`, input.name), + Data: input.upperOutput, + } + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.lowerStep, input.lowerStartTime, + []common.TestSeries{lowerExpected}, []*ts.Series{results.Values[0]}) + common.CompareOutputsAndExpected(t, input.upperStep, input.upperStartTime, + []common.TestSeries{upperExpected}, []*ts.Series{results.Values[1]}) + } +} + +func TestHoltWintersAberration(t *testing.T) { + ctx := common.NewTestContext() + ctx.Engine = NewEngine( + &mockStorage{}, + ) + defer ctx.Close() + + now := ctx.StartTime + tests := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + duration time.Duration + aberrationStartTime time.Time + aberrationStep int + aberrationOutput []float64 + }{ + { + "foo", + now, + 1000, + []float64{4.0, 5.0, 6.0}, + 3 * time.Second, + now, + 1000, + []float64{0, 0.7, 1.3298}, + }, + } + + for _, input := range tests { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + results, err := holtWintersAberrationInternal(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 3, input.duration) + expected := common.TestSeries{ + Name: fmt.Sprintf(`holtWintersAberration(%s)`, input.name), + Data: input.aberrationOutput, + } + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.aberrationStep, input.aberrationStartTime, + []common.TestSeries{expected}, results.Values) + } +} + +func TestSquareRoot(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + inputs := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + { + "foo", + startTime, + stepSize, + []float64{1.0, -2.0, 3.0, nan}, + }, + { + "bar", + startTime, + stepSize, + []float64{4.0}, + }, + } + + inputSeries := make([]*ts.Series, 0, len(inputs)) + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + inputSeries = append(inputSeries, series) + } + expected := []common.TestSeries{ + common.TestSeries{Name: "squareRoot(foo)", Data: []float64{1.0, nan, 1.73205, nan}}, + common.TestSeries{Name: "squareRoot(bar)", Data: []float64{2.0}}, + } + results, err := squareRoot(ctx, singlePathSpec{ + Values: inputSeries, + }) + require.Nil(t, err) + common.CompareOutputsAndExpected(t, stepSize, startTime, + expected, results.Values) +} + +func TestStdev(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + inputs := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + { + "foo", + startTime, + stepSize, + []float64{1.0, 2.0, 3.0, 4.0, nan, nan, nan, 5.0, 6.0, nan, nan}, + }, + } + + inputSeries := make([]*ts.Series, 0, len(inputs)) + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + inputSeries = append(inputSeries, series) + } + expected := []common.TestSeries{ + common.TestSeries{Name: "stddev(foo,3)", Data: []float64{0.0, 0.5, 0.8165, 0.8165, 0.5, 0.0, nan, 0.0, 0.5, 0.5, 0.0}}, + } + results, err := stdev(ctx, singlePathSpec{ + Values: inputSeries, + }, 3, 0.1) + require.Nil(t, err) + common.CompareOutputsAndExpected(t, stepSize, startTime, + expected, results.Values) +} + +func TestRangeOfSeries(t *testing.T) { + ctx, input := newConsolidationTestSeries() + defer ctx.Close() + + expectedStart := ctx.StartTime.Add(-30 * time.Second) + expectedStep := 10000 + rangeSeries, err := rangeOfSeries(ctx, singlePathSpec{ + Values: input, + }) + require.Nil(t, err) + expected := common.TestSeries{ + Name: "rangeOfSeries(a,b,c,d)", + Data: []float64{0, 0, 0, 12, 12, 12, 14, 14, 14, 0, 0, 0}, + } + common.CompareOutputsAndExpected(t, expectedStep, expectedStart, + []common.TestSeries{expected}, rangeSeries.Values) +} + +type percentileFunction func(ctx *common.Context, seriesList singlePathSpec, percentile float64) (ts.SeriesList, error) + +func testPercentileFunction(t *testing.T, f percentileFunction, expected []common.TestSeries) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + inputs := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + { + "foo", + startTime, + stepSize, + []float64{nan, nan, nan, nan, nan}, + }, + { + "bar", + startTime, + stepSize, + []float64{3.0, 2.0, 4.0, nan, 1.0, 6.0, nan, 5.0}, + }, + { + "baz", + startTime, + stepSize, + []float64{1.0}, + }, + } + + inputSeries := make([]*ts.Series, 0, len(inputs)) + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + inputSeries = append(inputSeries, series) + } + percentile := 40.123 + results, err := f(ctx, singlePathSpec{ + Values: inputSeries, + }, percentile) + require.Nil(t, err) + common.CompareOutputsAndExpected(t, stepSize, startTime, + expected, results.Values) +} + +func TestNPercentile(t *testing.T) { + expected := []common.TestSeries{ + common.TestSeries{ + Name: "nPercentile(bar, 40.123)", + Data: []float64{3.0, 3.0, 3.0, 3.0, 3.0, 3.0, 3.0, 3.0}, + }, + common.TestSeries{ + Name: "nPercentile(baz, 40.123)", + Data: []float64{1.0}, + }, + } + testPercentileFunction(t, nPercentile, expected) +} + +func TestRemoveAbovePercentile(t *testing.T) { + nan := math.NaN() + expected := []common.TestSeries{ + common.TestSeries{ + Name: "removeAbovePercentile(foo, 40.123)", + Data: []float64{nan, nan, nan, nan, nan}, + }, + common.TestSeries{ + Name: "removeAbovePercentile(bar, 40.123)", + Data: []float64{3.0, 2.0, nan, nan, 1.0, nan, nan, nan}, + }, + common.TestSeries{ + Name: "removeAbovePercentile(baz, 40.123)", + Data: []float64{1.0}, + }, + } + + testPercentileFunction(t, removeAbovePercentile, expected) +} + +func TestRemoveBelowPercentile(t *testing.T) { + nan := math.NaN() + + expected := []common.TestSeries{ + common.TestSeries{ + Name: "removeBelowPercentile(foo, 40.123)", + Data: []float64{nan, nan, nan, nan, nan}, + }, + common.TestSeries{ + Name: "removeBelowPercentile(bar, 40.123)", + Data: []float64{3.0, nan, 4.0, nan, nan, 6.0, nan, 5.0}, + }, + common.TestSeries{ + Name: "removeBelowPercentile(baz, 40.123)", + Data: []float64{1.0}, + }, + } + + testPercentileFunction(t, removeBelowPercentile, expected) +} + +func testRandomWalkFunctionInternal(t *testing.T, ctx *common.Context, stepSize, expectedLen int) { + r, err := randomWalkFunction(ctx, "foo", stepSize) + require.Nil(t, err) + + results := r.Values + require.Equal(t, 1, len(results)) + require.Equal(t, expectedLen, results[0].Len()) + for i := 0; i < expectedLen; i++ { + v := results[0].ValueAt(i) + require.True(t, v >= -0.5 && v < 0.5) + } +} + +func TestRandomWalkFunction(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + ctx.EndTime = ctx.StartTime.Add(1100 * time.Millisecond) + testRandomWalkFunctionInternal(t, ctx, 1, 2) + + ctx.EndTime = ctx.StartTime.Add(1600 * time.Millisecond) + testRandomWalkFunctionInternal(t, ctx, 1, 2) +} + +func testAggregateLineInternal(t *testing.T, f string, expectedName string, expectedVal float64) { + ctx := common.NewTestContext() + defer ctx.Close() + + input := struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + "foo", + ctx.StartTime, + 10000, + []float64{1.0, 2.0, 3.0, 4.0}, + } + + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + + r, err := aggregateLine(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, f) + require.Nil(t, err) + + results := r.Values + require.Equal(t, 1, len(results)) + require.Equal(t, expectedName, results[0].Name()) + require.Equal(t, 2, results[0].Len()) + for i := 0; i < 2; i++ { + require.Equal(t, expectedVal, results[0].ValueAt(i)) + } +} + +func TestAggregateLine(t *testing.T) { + testAggregateLineInternal(t, "avg", "aggregateLine(foo,2.500)", 2.5) + testAggregateLineInternal(t, "max", "aggregateLine(foo,4.000)", 4.0) + testAggregateLineInternal(t, "min", "aggregateLine(foo,1.000)", 1.0) +} + +func TestChanged(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + input := struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + "foo", + startTime, + stepSize, + []float64{1.0, 1.0, 2.0, 3.0, nan, 3.0, nan, 4.0, nan}, + } + + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + + expected := []common.TestSeries{ + common.TestSeries{ + Name: "changed(foo)", + Data: []float64{0.0, 0.0, 1.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0}, + }, + } + results, err := changed(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }) + require.Nil(t, err) + common.CompareOutputsAndExpected(t, stepSize, startTime, + expected, results.Values) +} + +// TODO: re-enable +// nolint +func testMovingMedian(t *testing.T) { + now := time.Now() + engine := NewEngine( + testStorage, + ) + startTime := now.Add(-3 * time.Minute) + endTime := now.Add(-time.Minute) + ctx := common.NewContext(common.ContextOptions{Start: startTime, End: endTime, Engine: engine}) + defer ctx.Close() + + stepSize := 60000 + target := "movingMedian(foo.bar.q.zed, '1min')" + expr, err := engine.Compile(target) + require.NoError(t, err) + res, err := expr.Execute(ctx) + require.NoError(t, err) + expected := common.TestSeries{ + Name: "movingMedian(foo.bar.q.zed,\"1min\")", + Data: []float64{0.0, 0.0}, + } + common.CompareOutputsAndExpected(t, stepSize, startTime, + []common.TestSeries{expected}, res.Values) +} + +func TestLegendValue(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + vals := []float64{1.0, 2.0, 3.0, 4.0, math.NaN()} + input := struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + "foo", + ctx.StartTime, + 10000, + vals, + } + + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + + results, err := legendValue(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, "avg") + expected := common.TestSeries{Name: "foo (avg: 2.500)", Data: vals} + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.stepInMilli, input.startTime, + []common.TestSeries{expected}, results.Values) + + results, err = legendValue(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, "last") + expected = common.TestSeries{Name: "foo (last: 4.000)", Data: vals} + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.stepInMilli, input.startTime, + []common.TestSeries{expected}, results.Values) + + results, err = legendValue(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, "unknown") + require.NotNil(t, err) +} + +func TestCactiStyle(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + stepSize := 10000 + inputs := []struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + { + "foo", + ctx.StartTime, + stepSize, + []float64{1.0, 2.0, 3.0, 4.0, math.NaN()}, + }, + { + "barbaz", + ctx.StartTime, + stepSize, + []float64{10.0, -5.0, 80.0, 100.0, math.NaN()}, + }, + { + "test", + ctx.StartTime, + stepSize, + []float64{math.NaN()}, + }, + } + + inputSeries := make([]*ts.Series, 0, len(inputs)) + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + inputSeries = append(inputSeries, series) + } + + results, err := cactiStyle(ctx, singlePathSpec{ + Values: inputSeries, + }) + expected := []common.TestSeries{ + {Name: "foo Current:4.00 Max:4.00 Min:1.00 ", Data: inputs[0].values}, + {Name: "barbaz Current:100.00 Max:100.00 Min:-5.00 ", Data: inputs[1].values}, + {Name: "test Current:nan Max:nan Min:nan ", Data: inputs[2].values}, + } + require.Nil(t, err) + common.CompareOutputsAndExpected(t, stepSize, ctx.StartTime, + expected, results.Values) +} + +func TestConsolidateBy(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + stepSize := 10000 + input := struct { + name string + startTime time.Time + stepInMilli int + values []float64 + }{ + "foo", + ctx.StartTime, + stepSize, + []float64{1.0, 2.0, 3.0, 4.0, math.NaN()}, + } + + series := ts.NewSeries( + ctx, + input.name, + input.startTime, + common.NewTestSeriesValues(ctx, input.stepInMilli, input.values), + ) + + results, err := consolidateBy(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, "min") + expected := common.TestSeries{Name: `consolidateBy(foo,"min")`, Data: input.values} + require.Nil(t, err) + common.CompareOutputsAndExpected(t, input.stepInMilli, input.startTime, + []common.TestSeries{expected}, results.Values) + + results, err = consolidateBy(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, "nonexistent") + require.NotNil(t, err) +} + +func TestOffsetToZero(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + inputs := []struct { + name string + values []float64 + expected []float64 + }{ + { + "foo", + []float64{nan, nan, nan, nan, nan}, + []float64{nan, nan, nan, nan, nan}, + }, + { + "bar", + []float64{3.0, 2.0, 4.0, nan, 1.0, 6.0, nan, 5.0}, + []float64{2.0, 1.0, 3.0, nan, 0.0, 5.0, nan, 4.0}, + }, + { + "baz", + []float64{1.0}, + []float64{0.0}, + }, + } + + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + startTime, + common.NewTestSeriesValues(ctx, stepSize, input.values), + ) + results, err := offsetToZero(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }) + require.NoError(t, err) + expected := common.TestSeries{ + Name: fmt.Sprintf("offsetToZero(%s)", input.name), + Data: input.expected, + } + common.CompareOutputsAndExpected(t, stepSize, startTime, + []common.TestSeries{expected}, results.Values) + } +} + +func TestTimeFunction(t *testing.T) { + ctx := common.NewTestContext() + now := time.Now() + truncatedNow := float64(now.Truncate(time.Second).Unix()) + ctx.StartTime = now + ctx.EndTime = now.Add(2 * time.Minute) + defer ctx.Close() + + results, err := timeFunction(ctx, "foo", 30) + require.NoError(t, err) + expected := common.TestSeries{ + Name: "foo", + Data: []float64{truncatedNow, truncatedNow + 30, truncatedNow + 60, truncatedNow + 90}, + } + common.CompareOutputsAndExpected(t, 30000, now.Truncate(time.Second), + []common.TestSeries{expected}, results.Values) +} + +// TODO arnikola reenable +// nolint +func testTimeShift(t *testing.T) { + now := time.Now() + engine := NewEngine( + testStorage, + ) + startTime := now.Add(-3 * time.Minute) + endTime := now.Add(-time.Minute) + ctx := common.NewContext(common.ContextOptions{ + Start: startTime, + End: endTime, + Engine: engine, + }) + defer ctx.Close() + + stepSize := 60000 + target := "timeShift(foo.bar.q.zed, '1min', false)" + expr, err := engine.Compile(target) + require.NoError(t, err) + res, err := expr.Execute(ctx) + require.NoError(t, err) + expected := common.TestSeries{ + Name: "timeShift(foo.bar.q.zed, -1min)", + Data: []float64{0.0, 0.0}, + } + common.CompareOutputsAndExpected(t, stepSize, startTime, + []common.TestSeries{expected}, res.Values) +} + +func TestDashed(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + nan := math.NaN() + startTime := ctx.StartTime + stepSize := 10000 + inputs := []struct { + name string + values []float64 + expected []float64 + }{ + { + "foo", + []float64{nan, nan, nan, nan, nan}, + []float64{nan, nan, nan, nan, nan}, + }, + } + + for _, input := range inputs { + series := ts.NewSeries( + ctx, + input.name, + startTime, + common.NewTestSeriesValues(ctx, stepSize, input.values), + ) + results, err := dashed(ctx, singlePathSpec{ + Values: []*ts.Series{series}, + }, 3.0) + require.NoError(t, err) + expected := common.TestSeries{ + Name: fmt.Sprintf("dashed(%s, 3.000)", input.name), + Data: input.expected, + } + common.CompareOutputsAndExpected(t, stepSize, startTime, + []common.TestSeries{expected}, results.Values) + } +} + +func TestThreshold(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + r, err := threshold(ctx, 1.0, "bar", "yellow") + require.NoError(t, err) + + results := r.Values + require.Equal(t, 1, len(results)) + require.Equal(t, "bar", results[0].Name()) + + r, err = threshold(ctx, 1.0, "", "red") + require.NoError(t, err) + + results = r.Values + require.Equal(t, 1, len(results)) + require.Equal(t, "1.000", results[0].Name()) + + r, err = threshold(ctx, 1.0, "", "") + require.NoError(t, err) + + results = r.Values + require.Equal(t, 1, len(results)) + require.Equal(t, "1.000", results[0].Name()) +} + +func TestFunctionsRegistered(t *testing.T) { + fnames := []string{ + "abs", + "absolute", + "aggregateLine", + "alias", + "aliasByMetric", + "aliasByNode", + "aliasSub", + "asPercent", + "averageAbove", + "averageSeries", + "averageSeriesWithWildcards", + "avg", + "cactiStyle", + "changed", + "consolidateBy", + "constantLine", + "countSeries", + "currentAbove", + "currentBelow", + "dashed", + "derivative", + "diffSeries", + "divideSeries", + "exclude", + "fallbackSeries", + "group", + "groupByNode", + "highestAverage", + "highestCurrent", + "highestMax", + "hitcount", + "holtWintersAberration", + "holtWintersConfidenceBands", + "holtWintersForecast", + "identity", + "integral", + "isNonNull", + "keepLastValue", + "legendValue", + "limit", + "log", + "logarithm", + "lowestAverage", + "lowestCurrent", + "max", + "maxSeries", + "maximumAbove", + "min", + "minSeries", + "minimumAbove", + "mostDeviant", + "movingAverage", + "movingMedian", + "multiplySeries", + "nonNegativeDerivative", + "nPercentile", + "offset", + "offsetToZero", + "perSecond", + "randomWalk", + "randomWalkFunction", + "rangeOfSeries", + "removeAbovePercentile", + "removeAboveValue", + "removeBelowPercentile", + "removeBelowValue", + "removeEmptySeries", + "scale", + "scaleToSeconds", + "sortByMaxima", + "sortByName", + "sortByTotal", + "squareRoot", + "stdev", + "substr", + "sum", + "sumSeries", + "summarize", + "threshold", + "time", + "timeFunction", + "timeShift", + "transformNull", + "weightedAverage", + } + + for _, fname := range fnames { + assert.NotNil(t, findFunction(fname), "could not find function: %s", fname) + } +} diff --git a/src/query/graphite/native/compiler.go b/src/query/graphite/native/compiler.go new file mode 100644 index 0000000000..9469ce4015 --- /dev/null +++ b/src/query/graphite/native/compiler.go @@ -0,0 +1,290 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "math" + "reflect" + "strconv" + + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/lexer" +) + +// compile converts an input stream into the corresponding Expression +func compile(input string) (Expression, error) { + booleanLiterals := map[string]lexer.TokenType{ + "true": lexer.True, + "false": lexer.False, + } + lex, tokens := lexer.NewLexer(input, booleanLiterals) + go lex.Run() + + c := compiler{input: input, tokens: tokens} + expr, err := c.compileExpression() + + // Exhaust all tokens until closed or else lexer won't close + for range tokens { + } + + return expr, err +} + +// A compiler converts an input string into an executable Expression +type compiler struct { + input string + tokens chan *lexer.Token +} + +// compileExpression compiles a top level expression +func (c *compiler) compileExpression() (Expression, error) { + token := <-c.tokens + if token == nil { + return noopExpression{}, nil + } + + var expr Expression + switch token.TokenType() { + case lexer.Pattern: + expr = newFetchExpression(token.Value()) + + case lexer.Identifier: + fc, err := c.compileFunctionCall(token.Value(), nil) + if err != nil { + return nil, err + } + + expr, err = newFuncExpression(fc) + if err != nil { + return nil, err + } + + default: + return nil, c.errorf("unexpected value %s", token.Value()) + } + + if token := <-c.tokens; token != nil { + return nil, c.errorf("extra data %s", token.Value()) + } + + return expr, nil +} + +// compileFunctionCall compiles a function call +func (c *compiler) compileFunctionCall(fname string, nextToken *lexer.Token) (*functionCall, error) { + fn := findFunction(fname) + if fn == nil { + return nil, c.errorf("could not find function named %s", fname) + } + + if nextToken != nil { + if nextToken.TokenType() != lexer.LParenthesis { + return nil, c.errorf("expected %v but encountered %s", lexer.LParenthesis, nextToken.Value()) + } + } else { + if _, err := c.expectToken(lexer.LParenthesis); err != nil { + return nil, err + } + } + + argTypes := fn.in + var args []funcArg + + // build up arguments for function call + for { + // if not variadic, function should be complete after reading len(argTypes) arguments + if !fn.variadic && len(args) == len(argTypes) { + _, err := c.expectToken(lexer.RParenthesis) + if err != nil { + return nil, err + } + break + } + + argType := argTypes[int(math.Min(float64(len(args)), float64(len(argTypes)-1)))] + nextArg, foundRParen, err := c.compileArg(fn.name, len(args), argType) + if err != nil { + return nil, err + } + if foundRParen { + break + } + + args = append(args, nextArg) + } + + // fill in defaults arguments for those not supplied by user explicitly + for len(args) < len(argTypes) { + defaultValue, ok := fn.defaults[uint8(len(args)+1)] + if !ok { + break + } + + args = append(args, newConstArg(defaultValue)) + } + + // all required argument types should be filled with values now + if len(args) < len(argTypes) { + variadicComment := "" + if fn.variadic { + variadicComment = "at least " + } + return nil, c.errorf("invalid number of arguments for %s; expected %s%d, received %d", + fn.name, variadicComment, len(argTypes), len(args)) + } + + return &functionCall{f: fn, in: args}, nil +} + +// compileArg parses and compiles a single argument +func (c *compiler) compileArg(fname string, index int, + reflectType reflect.Type) (arg funcArg, foundRParen bool, err error) { + token := <-c.tokens + if token == nil { + return nil, false, c.errorf("unexpected eof while parsing %s", fname) + } + + if token.TokenType() == lexer.RParenthesis { + return nil, true, nil + } + + if index > 0 { + if token.TokenType() != lexer.Comma { + return nil, false, c.errorf("error parsing %s expected ',' received '%s'", + fname, token.Value()) + } + + if token = <-c.tokens; token == nil { + return nil, false, c.errorf("unexpected eof while parsing %s", fname) + } + } + + arg, err = c.convertTokenToArg(token, reflectType) + if err != nil { + return nil, false, c.errorf("invalid function call %s, arg %d: %v", fname, index, err) + } + + if !arg.CompatibleWith(reflectType) { + return nil, false, c.errorf("invalid function call %s, arg %d: expected a %s, received '%s'", + fname, index, reflectType.Name(), arg) + } + + return arg, false, nil +} + +// convertTokenToArg converts the given token into the corresponding argument +func (c *compiler) convertTokenToArg(token *lexer.Token, reflectType reflect.Type) (funcArg, error) { + switch token.TokenType() { + case lexer.Number: + n, err := strconv.ParseFloat(token.Value(), 64) + if err != nil { + return nil, err + } + + if reflectType.Kind() == reflect.Int { + return newIntConst(int(n)), nil + } + + return newFloat64Const(n), nil + case lexer.String: + return newStringConst(token.Value()), nil + case lexer.Pattern: + return newFetchExpression(token.Value()), nil + case lexer.True, lexer.False: + b, err := strconv.ParseBool(token.Value()) + if err != nil { + return nil, err + } + return newBoolConst(b), nil + case lexer.Identifier: + currentToken := token.Value() + + // handle named arguments + nextToken := <-c.tokens + if nextToken == nil { + return nil, c.errorf("unexpected eof, %s should be followed by = or (", currentToken) + } + if nextToken.TokenType() == lexer.Equal { + // TODO: check if currentToken matches the expected parameter name + tokenAfterNext := <-c.tokens + if tokenAfterNext == nil { + return nil, c.errorf("unexpected eof, named argument %s should be followed by its value", currentToken) + } + return c.convertTokenToArg(tokenAfterNext, reflectType) + } + + return c.compileFunctionCall(currentToken, nextToken) + default: + return nil, c.errorf("%s not valid", token.Value()) + } +} + +// expectToken reads the next token and confirms it is the expected type before returning it +func (c *compiler) expectToken(expectedType lexer.TokenType) (*lexer.Token, error) { + token := <-c.tokens + if token == nil { + return nil, c.errorf("expected %v but encountered eof", expectedType) + } + + if token.TokenType() != expectedType { + return nil, c.errorf("expected %v but encountered %s", expectedType, token.Value()) + } + + return token, nil +} + +// errorf returns a formatted error vfrom the compiler +func (c *compiler) errorf(msg string, args ...interface{}) error { + return errors.NewInvalidParamsError(fmt.Errorf("invalid expression '%s': %s", c.input, fmt.Sprintf(msg, args...))) +} + +// ExtractFetchExpressions extracts timeseries fetch expressions from the given query +func ExtractFetchExpressions(s string) ([]string, error) { + expr, err := compile(s) + if err != nil { + return nil, err + } + + var targets []string + extractFetchExpressions(expr, &targets) + return targets, nil +} + +func extractFetchExpressions(expr Expression, targets *[]string) { + switch v := expr.(type) { + case *funcExpression: + extractFetchExpressionsFromFuncCall(v.call, targets) + case *fetchExpression: + *targets = append(*targets, v.pathArg.path) + } +} + +func extractFetchExpressionsFromFuncCall(call *functionCall, targets *[]string) { + for _, arg := range call.in { + switch varg := arg.(type) { + case *functionCall: + extractFetchExpressionsFromFuncCall(varg, targets) + case Expression: + extractFetchExpressions(varg, targets) + } + } +} diff --git a/src/query/graphite/native/compiler_test.go b/src/query/graphite/native/compiler_test.go new file mode 100644 index 0000000000..6149cd0408 --- /dev/null +++ b/src/query/graphite/native/compiler_test.go @@ -0,0 +1,450 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "math" + "testing" + + "github.com/m3db/m3/src/query/graphite/common" + xtest "github.com/m3db/m3/src/query/graphite/testing" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +type testCompile struct { + input string + result interface{} +} + +func hello(ctx *common.Context) (string, error) { return "hello", nil } +func noArgs(ctx *common.Context) (ts.SeriesList, error) { return ts.SeriesList{}, nil } +func defaultArgs(ctx *common.Context, b bool, f1, f2 float64, s string) (ts.SeriesList, error) { + return ts.SeriesList{}, nil +} + +func TestCompile1(t *testing.T) { + sortByName := findFunction("sortByName") + noArgs := findFunction("noArgs") + aliasByNode := findFunction("aliasByNode") + summarize := findFunction("summarize") + defaultArgs := findFunction("defaultArgs") + sumSeries := findFunction("sumSeries") + asPercent := findFunction("asPercent") + scale := findFunction("scale") + + tests := []testCompile{ + {"", noopExpression{}}, + {"foo.bar.{a,b,c}.baz-*.stat[0-9]", + newFetchExpression("foo.bar.{a,b,c}.baz-*.stat[0-9]")}, + {"noArgs()", &funcExpression{&functionCall{f: noArgs}}}, + {"sortByName(foo.bar.zed)", &funcExpression{ + &functionCall{ + f: sortByName, + in: []funcArg{ + newFetchExpression("foo.bar.zed"), + }, + }, + }}, + {"aliasByNode(foo.bar4.*.metrics.written, 2, 4)", &funcExpression{ + &functionCall{ + f: aliasByNode, + in: []funcArg{ + newFetchExpression("foo.bar4.*.metrics.written"), + newIntConst(2), + newIntConst(4), + }, + }, + }}, + {"summarize(foo.bar.baz.quux, \"1h\", \"max\", TRUE)", &funcExpression{ + &functionCall{ + f: summarize, + in: []funcArg{ + newFetchExpression("foo.bar.baz.quux"), + newStringConst("1h"), + newStringConst("max"), + newBoolConst(true), + }, + }, + }}, + {"summarize(foo.bar.baz.quuz, \"1h\")", &funcExpression{ + &functionCall{ + f: summarize, + in: []funcArg{ + newFetchExpression("foo.bar.baz.quuz"), + newStringConst("1h"), + newStringConst(""), + newBoolConst(false), + }, + }, + }}, + {"defaultArgs(true)", &funcExpression{ + &functionCall{ + f: defaultArgs, + in: []funcArg{ + newBoolConst(true), // non-default value + newFloat64Const(math.NaN()), // default value + newFloat64Const(100), // default value + newStringConst("foobar"), // default value + }, + }, + }}, + {"sortByName(aliasByNode(foo.bar72.*.metrics.written,2,4,6))", &funcExpression{ + &functionCall{ + f: sortByName, + in: []funcArg{ + &functionCall{ + f: aliasByNode, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + newIntConst(2), + newIntConst(4), + newIntConst(6), + }, + }, + }, + }, + }}, + {"sumSeries(foo.bar.baz.quux, foo.bar72.*.metrics.written)", &funcExpression{ + &functionCall{ + f: sumSeries, + in: []funcArg{ + newFetchExpression("foo.bar.baz.quux"), + newFetchExpression("foo.bar72.*.metrics.written"), + }, + }, + }}, + {"asPercent(foo.bar72.*.metrics.written, foo.bar.baz.quux)", &funcExpression{ + &functionCall{ + f: asPercent, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + newFetchExpression("foo.bar.baz.quux"), + }, + }, + }}, + {"asPercent(foo.bar72.*.metrics.written, sumSeries(foo.bar.baz.quux))", &funcExpression{ + &functionCall{ + f: asPercent, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + &functionCall{ + f: sumSeries, + in: []funcArg{ + newFetchExpression("foo.bar.baz.quux"), + }, + }, + }, + }, + }}, + {"asPercent(foo.bar72.*.metrics.written, 100)", &funcExpression{ + &functionCall{ + f: asPercent, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + newIntConst(100), + }, + }, + }}, + {"asPercent(foo.bar72.*.metrics.written)", &funcExpression{ + &functionCall{ + f: asPercent, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + newConstArg([]*ts.Series(nil)), + }, + }, + }}, + {"asPercent(foo.bar72.*.metrics.written, total=sumSeries(foo.bar.baz.quux))", &funcExpression{ + &functionCall{ + f: asPercent, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + &functionCall{ + f: sumSeries, + in: []funcArg{ + newFetchExpression("foo.bar.baz.quux"), + }, + }, + }, + }, + }}, + {"asPercent(foo.bar72.*.metrics.written, total=100)", &funcExpression{ + &functionCall{ + f: asPercent, + in: []funcArg{ + newFetchExpression("foo.bar72.*.metrics.written"), + newIntConst(100), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1e+3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(1000), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1e-3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(0.001), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1e3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(1000), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.1e3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(1100), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.1e+3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(1100), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.2e-3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(0.0012), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .1e+3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(100), + }, + }, + }}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 2.e+3)", &funcExpression{ + &functionCall{ + f: scale, + in: []funcArg{ + newFetchExpression("servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*"), + newFloat64Const(2000), + }, + }, + }}, + } + + for _, test := range tests { + expr, err := compile(test.input) + require.Nil(t, err, "error compiling: expression='%s', error='%v'", test.input, err) + require.NotNil(t, expr) + assertExprTree(t, test.result, expr, fmt.Sprintf("invalid result for %s: %v vs %v", + test.input, test.result, expr)) + } +} + +type testCompilerError struct { + input string + err string +} + +func TestCompileErrors(t *testing.T) { + tests := []testCompilerError{ + {"hello()", "top-level functions must return timeseries data"}, + {"sortByName(foo.*.zed)junk", "invalid expression 'sortByName(foo.*.zed)junk': " + + "extra data junk"}, + {"aliasByNode(", + "invalid expression 'aliasByNode(': unexpected eof while parsing aliasByNode"}, + {"unknownFunc()", + "invalid expression 'unknownFunc()': could not find function named unknownFunc"}, + {"aliasByNode(10)", + "invalid expression 'aliasByNode(10)': invalid function call aliasByNode," + + " arg 0: expected a singlePathSpec, received '10'"}, + {"sortByName(hello())", + "invalid expression 'sortByName(hello())': invalid function call " + + "sortByName, arg 0: expected a singlePathSpec, received 'hello()'"}, + {"aliasByNode()", + "invalid expression 'aliasByNode()': invalid number of arguments for aliasByNode;" + + " expected at least 2, received 0"}, + {"aliasByNode(foo.*.zed)", // check that at least 1 param is provided for variadic args + "invalid expression 'aliasByNode(foo.*.zed)': invalid number of arguments for " + + "aliasByNode; expected at least 2, received 1"}, + {"aliasByNode(foo.*.zed, 2, false)", + "invalid expression 'aliasByNode(foo.*.zed, 2, false)': invalid function call " + + "aliasByNode, arg 2: expected a int, received 'false'"}, + {"aliasByNode(foo.*.bar,", + "invalid expression 'aliasByNode(foo.*.bar,': unexpected eof while" + + " parsing aliasByNode"}, + {"aliasByNode(foo.*.bar,)", + "invalid expression 'aliasByNode(foo.*.bar,)': invalid function call" + + " aliasByNode, arg 1: invalid expression 'aliasByNode(foo.*.bar,)': ) not valid"}, + // TODO(jayp): Not providing all required parameters in a function with default parameters + // leads to an error message that states that a greater than required number of expected + // arguments. We could do better, but punting for now. + {"summarize(foo.bar.baz.quux)", + "invalid expression 'summarize(foo.bar.baz.quux)':" + + " invalid number of arguments for summarize; expected 4, received 1"}, + {"sumSeries()", // check that at least 1 series is provided for variadic timeSeriesList + "invalid expression 'sumSeries()': invalid number of arguments for sumSeries;" + + " expected at least 1, received 0"}, + {"sumSeries(foo.bar.baz.quux,)", + "invalid expression 'sumSeries(foo.bar.baz.quux,)': invalid function call sumSeries, " + + "arg 1: invalid expression 'sumSeries(foo.bar.baz.quux,)': ) not valid"}, + {"asPercent(foo.bar72.*.metrics.written, total", + "invalid expression 'asPercent(foo.bar72.*.metrics.written, total': " + + "invalid function call asPercent, " + + "arg 1: invalid expression 'asPercent(foo.bar72.*.metrics.written, total': " + + "unexpected eof, total should be followed by = or ("}, + {"asPercent(foo.bar72.*.metrics.written, total=", + "invalid expression 'asPercent(foo.bar72.*.metrics.written, total=': " + + "invalid function call asPercent, " + + "arg 1: invalid expression 'asPercent(foo.bar72.*.metrics.written, total=': " + + "unexpected eof, named argument total should be followed by its value"}, + {"asPercent(foo.bar72.*.metrics.written, total=randomStuff", + "invalid expression 'asPercent(foo.bar72.*.metrics.written, total=randomStuff': " + + "invalid function call asPercent, " + + "arg 1: invalid expression 'asPercent(foo.bar72.*.metrics.written, total=randomStuff': " + + "unexpected eof, randomStuff should be followed by = or ("}, + {"asPercent(foo.bar72.*.metrics.written, total=sumSeries(", + "invalid expression 'asPercent(foo.bar72.*.metrics.written, total=sumSeries(': " + + "invalid function call asPercent, " + + "arg 1: invalid expression 'asPercent(foo.bar72.*.metrics.written, total=sumSeries(': " + + "unexpected eof while parsing sumSeries"}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.e)", + "invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.e)': " + + "invalid function call scale, " + + "arg 1: invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.e)': " + + "expected one of 0123456789, found ) not valid"}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .1e)", + "invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .1e)': " + + "invalid function call scale, " + + "arg 1: invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .1e)': " + + "expected one of 0123456789, found ) not valid"}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .e)", + "invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .e)': " + + "invalid function call scale, " + + "arg 1: invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, .e)': " + + "expected one of 0123456789, found e not valid"}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, e)", + "invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, e)': " + + "invalid function call scale, " + + "arg 1: invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, e)': " + + "could not find function named e"}, + {"scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.2ee)", + "invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.2ee)': " + + "invalid function call scale, " + + "arg 1: invalid expression 'scale(servers.foobar*-qaz.quail.qux-qaz-qab.cpu.*, 1.2ee)': " + + "expected one of 0123456789, found e not valid"}, + } + + for _, test := range tests { + expr, err := compile(test.input) + require.NotNil(t, err, "no error for %s", test.input) + assert.Equal(t, test.err, err.Error(), "wrong error for %s", test.input) + assert.Nil(t, expr, "non-nil expression for %s", test.input) + } +} + +func assertExprTree(t *testing.T, expected interface{}, actual interface{}, msg string) { + switch e := expected.(type) { + case *functionCall: + a, ok := actual.(*functionCall) + require.True(t, ok, msg) + require.Equal(t, e.f.name, a.f.name, msg) + require.Equal(t, len(e.f.in), len(a.f.in), msg) + for i := range e.in { + assertExprTree(t, e.in[i], a.in[i], msg) + } + case noopExpression: + _, ok := actual.(noopExpression) + require.True(t, ok, msg) + case *funcExpression: + a, ok := actual.(*funcExpression) + require.True(t, ok, msg) + assertExprTree(t, e.call, a.call, msg) + case *fetchExpression: + a, ok := actual.(*fetchExpression) + require.True(t, ok, msg) + assert.Equal(t, e.pathArg.path, a.pathArg.path, msg) + case constFuncArg: + a, ok := actual.(constFuncArg) + require.True(t, ok, msg) + xtest.Equalish(t, e.value.Interface(), a.value.Interface(), msg) + default: + assert.Equal(t, expected, actual, msg) + } +} + +func TestExtractFetchExpressions(t *testing.T) { + tests := []struct { + expr string + targets []string + }{ + {"summarize(groupByNode(nonNegativeDerivative(foo.qaz.gauges.bar.baz.qux.foobar.*.quz.quail.count), 8, 'sum'), '10min', 'avg', true)", []string{ + "foo.qaz.gauges.bar.baz.qux.foobar.*.quz.quail.count", + }}, + {"asPercent(foo.bar72.*.metrics.written, total=sumSeries(foo.bar.baz.quux))", []string{ + "foo.bar72.*.metrics.written", "foo.bar.baz.quux", + }}, + {"foo.bar.{a,b,c}.baz-*.stat[0-9]", []string{ + "foo.bar.{a,b,c}.baz-*.stat[0-9]", + }}, + } + + for _, test := range tests { + targets, err := ExtractFetchExpressions(test.expr) + require.NoError(t, err) + assert.Equal(t, test.targets, targets, test.expr) + } + +} + +func init() { + MustRegisterFunction(noArgs) + MustRegisterFunction(hello) + MustRegisterFunction(defaultArgs).WithDefaultParams(map[uint8]interface{}{ + 1: false, + 2: math.NaN(), + 3: 100, + 4: "foobar", + }) +} diff --git a/src/query/graphite/native/engine.go b/src/query/graphite/native/engine.go new file mode 100644 index 0000000000..108da18a76 --- /dev/null +++ b/src/query/graphite/native/engine.go @@ -0,0 +1,65 @@ +// Copyright (c) 2019 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 native + +import ( + "time" + + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/storage" +) + +// The Engine for running queries. +type Engine struct { + storage storage.Storage +} + +// NewEngine creates a new query engine. +func NewEngine(store storage.Storage) *Engine { + return &Engine{ + storage: store, + } +} + +// FetchByQuery retrieves one or more time series based on a query. +func (e *Engine) FetchByQuery( + ctx context.Context, + query string, + start, end time.Time, + timeout time.Duration, +) (*storage.FetchResult, error) { + return e.storage.FetchByQuery( + ctx, + query, + storage.FetchOptions{ + StartTime: start, + EndTime: end, + DataOptions: storage.DataOptions{ + Timeout: timeout, + }, + }, + ) +} + +// Compile compiles an expression from an expression string +func (e *Engine) Compile(s string) (Expression, error) { + return compile(s) +} diff --git a/src/query/graphite/native/engine_test.go b/src/query/graphite/native/engine_test.go new file mode 100644 index 0000000000..d7d0ce20c5 --- /dev/null +++ b/src/query/graphite/native/engine_test.go @@ -0,0 +1,175 @@ +// Copyright (c) 2019 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 native + +import ( + "testing" + "time" + + "github.com/m3db/m3/src/metrics/policy" + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/storage" + xtime "github.com/m3db/m3x/time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// nolint +type queryTestResult struct { + name string + max float64 +} + +// nolint +type queryTest struct { + query string + ordered bool + results []queryTestResult +} + +var ( + // nolint + testValues = map[string]float64{ + "foo.bar.q.zed": 0, + "foo.bar.g.zed": 1, + "foo.bar.x.zed": 2, + "san_francisco.cake": 3, + "new_york_city.cake": 4, + "chicago.cake": 5, + "los_angeles.cake": 6, + } + + // nolint + testPolicy = policy.NewStoragePolicy(10*time.Second, xtime.Second, 48*time.Hour) + // testTSDB = makeTSDB(testPolicy) + // nolint + testStorage storage.Storage //= nil + // local.NewLocalStorage(local.Options{ + // Database: testTSDB, + // Workers: workers, + // Scope: metrics.None, + // PolicyResolver: resolver.NewStaticResolver(testIndex, testPolicy), + // }) +) + +// TODO arnikola reenable +// nolint +func testExecute(t *testing.T) { + engine := NewEngine( + testStorage, + ) + tests := []queryTest{ + {"foo.bar.q.zed", true, []queryTestResult{{"foo.bar.q.zed", 0}}}, + {"foo.bar.*.zed", false, []queryTestResult{ + {"foo.bar.q.zed", 0}, + {"foo.bar.g.zed", 1}, + {"foo.bar.x.zed", 2}}, + }, + {"sortByName(aliasByNode(foo.bar.*.zed, 0, 2))", true, []queryTestResult{ + {"foo.g", 1}, + {"foo.q", 0}, + {"foo.x", 2}, + }}, + } + + ctx := common.NewContext(common.ContextOptions{Start: time.Now().Add(-1 * time.Hour), End: time.Now(), Engine: engine}) + for _, test := range tests { + expr, err := engine.Compile(test.query) + require.Nil(t, err) + + results, err := expr.Execute(ctx) + require.Nil(t, err, "failed to execute %s", test.query) + require.Equal(t, len(test.results), len(results.Values), "invalid results for %s", test.query) + + for i := range test.results { + if test.ordered { + assert.Equal(t, test.results[i].name, results.Values[i].Name(), + "invalid result %d for %s", i, test.query) + assert.Equal(t, test.results[i].max, results.Values[i].CalcStatistics().Max, + "invalid result %d for %s", i, test.query) + } + } + } +} + +// TODO arnikola reenable +// nolint +func testTracing(t *testing.T) { + engine := NewEngine( + testStorage, + ) + var traces []common.Trace + + ctx := common.NewContext(common.ContextOptions{Start: time.Now().Add(-1 * time.Hour), End: time.Now(), Engine: engine}) + ctx.Trace = func(t common.Trace) { + traces = append(traces, t) + } + + expr, err := engine.Compile("groupByNode(sortByName(aliasByNode(foo.bar.*.zed, 0, 2)), 0, 'sumSeries')") + require.NoError(t, err) + + _, err = expr.Execute(ctx) + require.NoError(t, err) + + expectedTraces := []common.Trace{ + common.Trace{ + ActivityName: "fetch foo.bar.*.zed", + Outputs: common.TraceStats{NumSeries: 3}}, + common.Trace{ + ActivityName: "aliasByNode", + Inputs: []common.TraceStats{common.TraceStats{NumSeries: 3}}, + Outputs: common.TraceStats{NumSeries: 3}}, + common.Trace{ + ActivityName: "sortByName", + Inputs: []common.TraceStats{common.TraceStats{NumSeries: 3}}, + Outputs: common.TraceStats{NumSeries: 3}}, + common.Trace{ + ActivityName: "groupByNode", + Inputs: []common.TraceStats{common.TraceStats{NumSeries: 3}}, + Outputs: common.TraceStats{NumSeries: 1}}, + } + require.Equal(t, len(expectedTraces), len(traces)) + for i, expected := range expectedTraces { + trace := traces[i] + assert.Equal(t, expected.ActivityName, trace.ActivityName, "incorrect name for trace %d", i) + assert.Equal(t, expected.Inputs, trace.Inputs, "incorrect inputs for trace %d", i) + assert.Equal(t, expected.Outputs, trace.Outputs, "incorrect outputs for trace %d", i) + } +} + +// func makeTSDB(policy policy.StoragePolicy) tsdb.Database { +// var ( +// now = time.Now().Truncate(time.Second * 10) +// testTSDB = nil //FIXME mocktsdb.New() +// ctx = context.New() +// ) + +// defer ctx.Close() + +// for name, val := range testValues { +// for t := now.Add(-time.Hour * 2); t.Before(now.Add(time.Hour)); t = t.Add(time.Second * 10) { +// testTSDB.WriteRaw(ctx, name, t, val, policy) +// } +// } + +// return testIndex, testTSDB +// } diff --git a/src/query/graphite/native/expression.go b/src/query/graphite/native/expression.go new file mode 100644 index 0000000000..6dacfde4ab --- /dev/null +++ b/src/query/graphite/native/expression.go @@ -0,0 +1,181 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "reflect" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +var ( + errTopLevelFunctionMustReturnTimeSeries = errors.NewInvalidParamsError(errors.New("top-level functions must return timeseries data")) +) + +// An Expression is a metric query expression +type Expression interface { + CallASTNode + // Executes the expression against the given context, and returns the resulting time series data + Execute(ctx *common.Context) (ts.SeriesList, error) +} + +// CallASTNode is an interface to help with printing the AST. +type CallASTNode interface { + // Name returns the name of the call. + Name() string + // Arguments describe each argument that the call has, some + // arguments can be casted to an Call themselves. + Arguments() []ArgumentASTNode +} + +// ArgumentASTNode is an interface to help with printing the AST. +type ArgumentASTNode interface { + String() string +} + +// A fetchExpression is an expression that fetches a bunch of data from storage based on a path expression +type fetchExpression struct { + // The path expression to fetch + pathArg fetchExpressionPathArg +} + +type fetchExpressionPathArg struct { + path string +} + +func (a fetchExpressionPathArg) String() string { + return a.path +} + +// newFetchExpression creates a new fetch expression for a single path +func newFetchExpression(path string) *fetchExpression { + return &fetchExpression{pathArg: fetchExpressionPathArg{path: path}} +} + +func (f *fetchExpression) Name() string { + return "fetch" +} + +func (f *fetchExpression) Arguments() []ArgumentASTNode { + return []ArgumentASTNode{f.pathArg} +} + +// Execute fetches results from storage +func (f *fetchExpression) Execute(ctx *common.Context) (ts.SeriesList, error) { + begin := time.Now() + + result, err := ctx.Engine.FetchByQuery(ctx, f.pathArg.path, ctx.StartTime, + ctx.EndTime, ctx.Timeout) + if err != nil { + return ts.SeriesList{}, err + } + + if ctx.TracingEnabled() { + ctx.Trace(common.Trace{ + ActivityName: fmt.Sprintf("fetch %s", f.pathArg.path), + Duration: time.Since(begin), + Outputs: common.TraceStats{NumSeries: len(result.SeriesList)}, + }) + } + + for _, r := range result.SeriesList { + r.Specification = f.pathArg.path + } + return ts.SeriesList{Values: result.SeriesList}, nil +} + +// Evaluate evaluates the fetch and returns its results as a reflection value, allowing it to be used +// as an input argument to a function that takes a time series +func (f *fetchExpression) Evaluate(ctx *common.Context) (reflect.Value, error) { + timeseries, err := f.Execute(ctx) + if err != nil { + return reflect.Value{}, err + } + + return reflect.ValueOf(timeseries), nil +} + +// CompatibleWith returns true if the reflected type is a time series or a generic interface. +func (f *fetchExpression) CompatibleWith(reflectType reflect.Type) bool { + return reflectType == singlePathSpecType || reflectType == multiplePathSpecsType || reflectType == interfaceType +} + +func (f *fetchExpression) String() string { + return fmt.Sprintf("fetch(%s)", f.pathArg.path) +} + +// A funcExpression is an expression that evaluates a function returning a timeseries +type funcExpression struct { + call *functionCall +} + +// newFuncExpression creates a new expressioon based on the given function call +func newFuncExpression(call *functionCall) (Expression, error) { + if !(call.f.out == seriesListType || call.f.out == unaryContextShifterPtrType || call.f.out == binaryContextShifterPtrType) { + return nil, errTopLevelFunctionMustReturnTimeSeries + } + + return &funcExpression{call: call}, nil +} + +func (f *funcExpression) Name() string { + return f.call.Name() +} + +func (f *funcExpression) Arguments() []ArgumentASTNode { + return f.call.Arguments() +} + +// Execute evaluates the function and returns the result as a timeseries +func (f *funcExpression) Execute(ctx *common.Context) (ts.SeriesList, error) { + out, err := f.call.Evaluate(ctx) + if err != nil { + return ts.SeriesList{}, err + } + + return out.Interface().(ts.SeriesList), nil +} + +func (f *funcExpression) String() string { return f.call.String() } + +// A noopExpression is an empty expression that returns nothing +type noopExpression struct{} + +// Execute returns nothing +func (noop noopExpression) Execute(ctx *common.Context) (ts.SeriesList, error) { + return ts.SeriesList{}, nil +} + +func (noop noopExpression) Name() string { + return "noop" +} + +func (noop noopExpression) Arguments() []ArgumentASTNode { + return nil +} + +func (noop noopExpression) String() string { + return noop.Name() +} diff --git a/src/query/graphite/native/functions.go b/src/query/graphite/native/functions.go new file mode 100644 index 0000000000..b7859da30e --- /dev/null +++ b/src/query/graphite/native/functions.go @@ -0,0 +1,576 @@ +// Copyright (c) 2019 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 native + +import ( + "bytes" + "fmt" + "reflect" + "runtime" + "strings" + "sync" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +var ( + funcMut sync.RWMutex + functions = map[string]*Function{} +) + +// registerFunction is used to register a function under a specific name +func registerFunction(f interface{}) (*Function, error) { + fn, err := buildFunction(f) + if err != nil { + return nil, err + } + + funcMut.Lock() + defer funcMut.Unlock() + + if functions[fn.name] != nil { + return nil, fmt.Errorf("func %s already registered", fn.name) + } + functions[fn.name] = fn + return fn, nil +} + +// MustRegisterFunction registers a function, issuing a panic if the function cannot be registered +func MustRegisterFunction(f interface{}) *Function { + if fn, err := registerFunction(f); err != nil { + if name, nerr := functionName(f); nerr == nil { + err = fmt.Errorf("could not register %s: %v", name, err) + } + panic(err) + } else { + return fn + } +} + +// registerAliasedFunction is used to register a function under an alias +func registerAliasedFunction(alias string, f interface{}) error { + fname, err := functionName(f) + if err != nil { + return err + } + + funcMut.Lock() + defer funcMut.Unlock() + + if functions[alias] != nil { + return fmt.Errorf("func %s already registered", alias) + } + + fn := functions[fname] + if fn == nil { + return fmt.Errorf("target function %s not registered", fname) + } + + functions[alias] = fn + return nil +} + +// MustRegisterAliasedFunction registers a function under an alias, issuing a panic if the function +// cannot be registered +func MustRegisterAliasedFunction(fname string, f interface{}) { + if err := registerAliasedFunction(fname, f); err != nil { + panic(err) + } +} + +// findFunction finds a function with the given name +func findFunction(name string) *Function { + funcMut.RLock() + defer funcMut.RUnlock() + + return functions[name] +} + +// reflectTypeSet is a set of reflect.Type objects +type reflectTypeSet []reflect.Type + +// contains checks whether the type set contains the given type +func (ts reflectTypeSet) contains(reflectType reflect.Type) bool { + for i := range ts { + if ts[i] == reflectType { + return true + } + } + + return false +} + +// singlePathSpec represents one wildcard pathspec argument that may fetch multiple time series +type singlePathSpec ts.SeriesList + +// multiplePathSpecs represents a variadic number of wildcard pathspecs +type multiplePathSpecs ts.SeriesList + +// genericInterface represents a value with an arbitrary type +type genericInterface interface{} + +// contextShiftFunc generates a shifted context based on an input context +type contextShiftFunc func(*common.Context) *common.Context + +// unaryTransformer takes in one series and returns a transformed series. +type unaryTransformer func(ts.SeriesList) (ts.SeriesList, error) + +// binaryTransformer takes in two series and returns a transformed series. +type binaryTransformer func(ts.SeriesList, ts.SeriesList) (ts.SeriesList, error) + +// unaryContextShifter contains a contextShiftFunc for generating shift contexts +// as well as a unaryTransformer for transforming one series to another. +type unaryContextShifter struct { + ContextShiftFunc contextShiftFunc + UnaryTransformer unaryTransformer +} + +// binaryContextShifter contains a contextShiftFunc for generating shift contexts +// as well as a binaryTransformer for transforming one series to another. +type binaryContextShifter struct { + ContextShiftFunc contextShiftFunc + BinaryTransformer binaryTransformer +} + +var ( + contextPtrType = reflect.TypeOf(&common.Context{}) + timeSeriesType = reflect.TypeOf(&ts.Series{}) + timeSeriesListType = reflect.SliceOf(timeSeriesType) + seriesListType = reflect.TypeOf(ts.SeriesList{}) + unaryContextShifterPtrType = reflect.TypeOf(&unaryContextShifter{}) + binaryContextShifterPtrType = reflect.TypeOf(&binaryContextShifter{}) + singlePathSpecType = reflect.TypeOf(singlePathSpec{}) + multiplePathSpecsType = reflect.TypeOf(multiplePathSpecs{}) + interfaceType = reflect.TypeOf([]genericInterface{}).Elem() + float64Type = reflect.TypeOf(float64(100)) + float64SliceType = reflect.SliceOf(float64Type) + intType = reflect.TypeOf(int(0)) + intSliceType = reflect.SliceOf(intType) + stringType = reflect.TypeOf("") + stringSliceType = reflect.SliceOf(stringType) + boolType = reflect.TypeOf(false) + boolSliceType = reflect.SliceOf(boolType) + errorType = reflect.TypeOf((*error)(nil)).Elem() + genericInterfaceType = reflect.TypeOf((*genericInterface)(nil)).Elem() +) + +var ( + allowableTypes = reflectTypeSet{ + // these are for return types + timeSeriesListType, + unaryContextShifterPtrType, + binaryContextShifterPtrType, + seriesListType, + singlePathSpecType, + multiplePathSpecsType, + interfaceType, // only for function parameters + float64Type, + float64SliceType, + intType, + intSliceType, + stringType, + stringSliceType, + boolType, + boolSliceType, + } +) + +var ( + errNonFunction = errors.NewInvalidParamsError(errors.New("not a function")) + errNeedsArgument = errors.NewInvalidParamsError(errors.New("functions must take at least 1 argument")) + errNoContext = errors.NewInvalidParamsError(errors.New("first argument must be a context")) + errInvalidReturn = errors.NewInvalidParamsError(errors.New("functions must return a value and an error")) +) + +// Function contains a function to invoke along with metadata about +// the function's argument and return type. +type Function struct { + name string + f reflect.Value + in []reflect.Type + defaults map[uint8]interface{} + out reflect.Type + variadic bool +} + +// WithDefaultParams provides default parameters for functions +func (f *Function) WithDefaultParams(defaultParams map[uint8]interface{}) *Function { + for index := range defaultParams { + if int(index) <= 0 || int(index) > len(f.in) { + panic(fmt.Sprintf("Default parameter #%d is out-of-range", index)) + } + } + f.defaults = defaultParams + return f +} + +func functionName(f interface{}) (string, error) { + v := reflect.ValueOf(f) + t := v.Type() + if t.Kind() != reflect.Func { + return "", errNonFunction + } + + nameParts := strings.Split(runtime.FuncForPC(v.Pointer()).Name(), ".") + return nameParts[len(nameParts)-1], nil +} + +// validateContextShiftingFn validates if a function is a context shifting function. +func validateContextShiftingFn(in []reflect.Type) { + // check that we have exactly *one* singlePathSpec parameter + singlePathSpecParams := 0 + singlePathSpecIndex := -1 + for i, param := range in { + if param == singlePathSpecType { + singlePathSpecParams++ + singlePathSpecIndex = i + } + } + if singlePathSpecParams != 1 { + panic("A context-shifting function must have exactly one singlePathSpec parameter") + } + if singlePathSpecIndex != 0 { + panic("A context-shifting function must have the singlePathSpec parameter as its first parameter") + } +} + +// buildFunction takes a reflection reference to a function and returns +// the function metadata +func buildFunction(f interface{}) (*Function, error) { + fname, err := functionName(f) + if err != nil { + return nil, err + } + v := reflect.ValueOf(f) + t := v.Type() + if t.NumIn() == 0 { + return nil, errNeedsArgument + } + + if ctx := t.In(0); ctx != contextPtrType { + return nil, errNoContext + } + + var lastType reflect.Type + in := make([]reflect.Type, 0, t.NumIn()-1) + for i := 1; i < t.NumIn(); i++ { + inArg := t.In(i) + if !(allowableTypes.contains(inArg)) { + return nil, fmt.Errorf("invalid arg %d: %s is not supported", i, inArg.Name()) + } + if inArg == multiplePathSpecsType && i != t.NumIn()-1 { + return nil, fmt.Errorf("invalid arg %d: multiplePathSpecs must be the last arg", i) + } + + lastType = inArg + in = append(in, inArg) + } + + variadic := lastType == multiplePathSpecsType || + (lastType != nil && + lastType.Kind() == reflect.Slice && + lastType != singlePathSpecType) + + if variadic { // remove slice-ness of the variadic arg + if lastType != multiplePathSpecsType { + in[len(in)-1] = in[len(in)-1].Elem() + } + } + + if t.NumOut() != 2 { + return nil, errInvalidReturn + } + + out := t.Out(0) + if !allowableTypes.contains(out) { + return nil, fmt.Errorf("invalid return type %s", out.Name()) + } else if out == unaryContextShifterPtrType || out == binaryContextShifterPtrType { + validateContextShiftingFn(in) + } + + if ret2 := t.Out(1); ret2 != errorType { + return nil, errInvalidReturn + } + + return &Function{ + name: fname, + f: v, + in: in, + out: out, + variadic: variadic, + }, nil +} + +// call calls the function with non-reflected values +func (f *Function) call(ctx *common.Context, args []interface{}) (interface{}, error) { + values := make([]reflect.Value, len(args)) + for i := range args { + values[i] = reflect.ValueOf(args[i]) + } + + out, err := f.reflectCall(ctx, values) + if err != nil { + return nil, err + } + + return out.Interface(), err +} + +// reflectCall calls the function with reflected values, passing in the provided context and parameters +func (f *Function) reflectCall(ctx *common.Context, args []reflect.Value) (reflect.Value, error) { + var instats []common.TraceStats + + in := make([]reflect.Value, 0, len(args)+1) + in = append(in, reflect.ValueOf(ctx)) + for _, arg := range args { + in = append(in, arg) + if isTimeSeries(arg) { + instats = append(instats, getStats(arg)) + } + } + + // special case handling of multiplePathSpecs + // NB(r): This code sucks, and it would be better if we just removed + // multiplePathSpecs altogether and have the functions use real variadic + // ts.SeriesList arguments so we don't have to autocollapse when calling here. + // Notably singlePathSpec should also go and just replace usages with + // barebones ts.SeriesList. Then we can get rid of this code below and + // the code the casts ts.SeriesList to the correct typealias of ts.SeriesList. + if len(in) > len(f.in)+1 && len(f.in) > 0 && f.in[len(f.in)-1] == multiplePathSpecsType { + var ( + series = make([]*ts.Series, 0, len(in)) + // Assume all sorted until proven otherwise + sortedAll = true + ) + for i := len(f.in); i < len(in); i++ { + v := in[i].Interface().(ts.SeriesList) + + // If any series lists are not sorted then the result + // is not in deterministic sort order + if sortedAll && !v.SortApplied { + sortedAll = false + } + + series = append(series, v.Values...) + } + + in[len(f.in)] = reflect.ValueOf(ts.SeriesList{ + Values: series, + // Only consider the aggregation of all these series lists + // sorted if and only if all originally had a sort applied + SortApplied: sortedAll, + }) + + in = in[:len(f.in)+1] + } + + numTypes := len(f.in) + if len(in) < numTypes { + err := fmt.Errorf("call args mismatch: expected at least %d, actual %d", + len(f.in), len(in)) + return reflect.Value{}, err + } + + // Cast to the expected typealias type of ts.SeriesList before calling + for i, arg := range in { + typeArg := arg.Type() + if typeArg != seriesListType { + continue + } + // NB(r): Poor form, ctx is not in f.in for no reason it seems... + typeIdx := i - 1 + if i >= numTypes { + typeIdx = numTypes - 1 + } + l := arg.Interface().(ts.SeriesList) + switch f.in[typeIdx] { + case singlePathSpecType, genericInterfaceType: + in[i] = reflect.ValueOf(singlePathSpec(l)) + case multiplePathSpecsType: + in[i] = reflect.ValueOf(multiplePathSpecs(l)) + default: + err := fmt.Errorf("cannot cast series to unexpected type: %s", + f.in[typeIdx].String()) + return reflect.Value{}, err + } + } + + beginCall := time.Now() + out := f.f.Call(in) + outVal, errVal := out[0], out[1] + var err error + if !errVal.IsNil() { + err = errVal.Interface().(error) + return outVal, err + } + + if ctx.TracingEnabled() { + var outstats common.TraceStats + if isTimeSeries(outVal) { + outstats = getStats(outVal) + } + + ctx.Trace(common.Trace{ + ActivityName: f.name, + Inputs: instats, + Outputs: outstats, + Duration: time.Since(beginCall), + }) + } + + return outVal, nil +} + +// A funcArg is an argument to a function that gets resolved at runtime +type funcArg interface { + ArgumentASTNode + Evaluate(ctx *common.Context) (reflect.Value, error) + CompatibleWith(reflectType reflect.Type) bool +} + +// A constFuncArg is a function argument that is a constant value +type constFuncArg struct { + value reflect.Value +} + +func newConstArg(i interface{}) funcArg { return constFuncArg{value: reflect.ValueOf(i)} } +func newBoolConst(b bool) funcArg { return constFuncArg{value: reflect.ValueOf(b)} } +func newStringConst(s string) funcArg { return constFuncArg{value: reflect.ValueOf(s)} } +func newFloat64Const(n float64) funcArg { return constFuncArg{value: reflect.ValueOf(n)} } +func newIntConst(n int) funcArg { return constFuncArg{value: reflect.ValueOf(n)} } + +func (c constFuncArg) Evaluate(ctx *common.Context) (reflect.Value, error) { return c.value, nil } +func (c constFuncArg) CompatibleWith(reflectType reflect.Type) bool { + return c.value.Type() == reflectType || reflectType == interfaceType +} +func (c constFuncArg) String() string { return fmt.Sprintf("%v", c.value.Interface()) } + +// A functionCall is an actual call to a function, with resolution for arguments +type functionCall struct { + f *Function + in []funcArg +} + +func (call *functionCall) Name() string { + return call.f.name +} + +func (call *functionCall) Arguments() []ArgumentASTNode { + args := make([]ArgumentASTNode, len(call.in)) + for i, arg := range call.in { + args[i] = arg + } + return args +} + +// Evaluate evaluates the function call and returns the result as a reflect.Value +func (call *functionCall) Evaluate(ctx *common.Context) (reflect.Value, error) { + values := make([]reflect.Value, len(call.in)) + for i, param := range call.in { + if call.f.out == unaryContextShifterPtrType && call.f.in[i] == singlePathSpecType { + values[i] = reflect.ValueOf(singlePathSpec{}) // fake parameter + continue + } + value, err := param.Evaluate(ctx) + if err != nil { + return reflect.Value{}, err + } + values[i] = value + } + + result, err := call.f.reflectCall(ctx, values) + // if we have errors, or if we succeed and this is not a context-shifting function, + // we return immediately + if err != nil || call.f.out == seriesListType { + return result, err + } + + // context shifter ptr is nil, nothing to do here, return empty series. + if result.IsNil() { + return reflect.ValueOf(ts.SeriesList{}), nil + } + + contextShifter := result.Elem() + ctxShiftingFn := contextShifter.Field(0) + reflected := ctxShiftingFn.Call([]reflect.Value{reflect.ValueOf(ctx)}) + shiftedCtx := reflected[0].Interface().(*common.Context) + shiftedSeries, err := call.in[0].Evaluate(shiftedCtx) + if err != nil { + return reflect.Value{}, err + } + transformerFn := contextShifter.Field(1) + var ret []reflect.Value + if call.f.out == unaryContextShifterPtrType { + // unary function + ret = transformerFn.Call([]reflect.Value{shiftedSeries}) + } else { + ret = transformerFn.Call([]reflect.Value{shiftedSeries, values[0]}) + } + if !ret[1].IsNil() { + err = ret[1].Interface().(error) + } + return ret[0], err +} + +// CompatibleWith checks whether the function call's return is compatible with the given reflection type +func (call *functionCall) CompatibleWith(reflectType reflect.Type) bool { + if reflectType == interfaceType { + return true + } + if call.f.out == unaryContextShifterPtrType || call.f.out == binaryContextShifterPtrType { + return reflectType == singlePathSpecType || reflectType == multiplePathSpecsType + } + return call.f.out.Kind() == reflectType.Kind() +} + +func (call *functionCall) String() string { + var buf bytes.Buffer + buf.WriteString(call.f.name) + buf.WriteByte('(') + for i := range call.in { + if i > 0 { + buf.WriteByte(',') + } + buf.WriteString(call.in[i].String()) + } + + buf.WriteByte(')') + return buf.String() +} + +// isTimeSeries checks whether the given value contains a timeseries or +// timeseries list +func isTimeSeries(v reflect.Value) bool { + return v.Type() == seriesListType +} + +// getStats gets trace stats for the given timeseries argument +func getStats(v reflect.Value) common.TraceStats { + if v.Type() == timeSeriesType { + return common.TraceStats{NumSeries: 1} + } + + l := v.Interface().(ts.SeriesList) + return common.TraceStats{NumSeries: l.Len()} +} diff --git a/src/query/graphite/native/functions_test.go b/src/query/graphite/native/functions_test.go new file mode 100644 index 0000000000..9b1fb1f878 --- /dev/null +++ b/src/query/graphite/native/functions_test.go @@ -0,0 +1,125 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "reflect" + "sort" + "testing" + + "github.com/m3db/m3/src/query/graphite/common" + xtest "github.com/m3db/m3/src/query/graphite/testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func f1(ctx *common.Context, a float64, b string, c bool) (string, error) { + return fmt.Sprintf("%.3f %s %t", a, b, c), nil +} + +func f2(ctx *common.Context, a ...string) ([]string, error) { + sort.Strings(a) + return a, nil +} + +func f3(ctx *common.Context, values ...float64) (float64, error) { + sum := float64(0) + for _, n := range values { + sum += n + } + + return sum, nil +} + +type testFunction struct { + f interface{} + input []interface{} + expectedOutput interface{} +} + +var testFunctions = []testFunction{ + {f1, []interface{}{635.6, "Hello", false}, "635.600 Hello false"}, + {f2, []interface{}{"b", "c", "a"}, []string{"a", "b", "c"}}, + {f3, []interface{}{10.0, 20.0, 30.0}, 60}, +} + +func TestFunctions(t *testing.T) { + ctx := common.NewTestContext() + defer ctx.Close() + + for _, tf := range testFunctions { + f, err := buildFunction(tf.f) + require.Nil(t, err, "could not build function %s", reflect.TypeOf(tf.f).Name()) + + out, err := f.call(ctx, tf.input) + require.Nil(t, err, "Could not call function %s", reflect.TypeOf(tf.f).Name()) + xtest.Equalish(t, tf.expectedOutput, out) + } +} + +func errorf(ctx *common.Context) ([]float64, error) { + return nil, fmt.Errorf("this failed") +} + +func TestFunctionReturningError(t *testing.T) { + f, err := buildFunction(errorf) + require.Nil(t, err) + + _, err = f.call(nil, nil) + require.NotNil(t, err) + assert.Equal(t, "this failed", err.Error()) +} + +type invalidFunction struct { + name string + f interface{} + expectedError string +} + +func badf1() (float64, error) { return 0, nil } +func badf2(ctx *common.Context) {} +func badf3(ctx *common.Context) (float32, error) { return 0, nil } +func badf4(ctx *common.Context) (string, string) { return "", "" } +func badf5(ctx *common.Context, n byte) (string, error) { return "", nil } +func badf6(ctx *common.Context, n float64) (byte, error) { return 0, nil } +func badf7(ctx *common.Context, foo, bar multiplePathSpecs) (string, error) { return "", nil } + +func TestInvalidFunctions(t *testing.T) { + invalidFunctions := []invalidFunction{ + {"badf1", badf1, "functions must take at least 1 argument"}, + {"badf2", badf2, "functions must return a value and an error"}, + {"badf3", badf3, "invalid return type float32"}, + {"badf4", badf4, "functions must return a value and an error"}, + {"badf5", badf5, "invalid arg 1: uint8 is not supported"}, + {"badf6", badf6, "invalid return type uint8"}, + {"24", 24, "not a function"}, + {"badf7", badf7, "invalid arg 1: multiplePathSpecs must be the last arg"}, + } + + for i, fn := range invalidFunctions { + f, err := buildFunction(fn.f) + require.NotNil(t, err, "invalid error for %s (%d)", fn.name, i) + assert.Equal(t, fn.expectedError, err.Error(), "invalid error for %s (%d)", fn.name, i) + assert.Nil(t, f) + } +} diff --git a/src/query/graphite/native/summarize.go b/src/query/graphite/native/summarize.go new file mode 100644 index 0000000000..283c7d3004 --- /dev/null +++ b/src/query/graphite/native/summarize.go @@ -0,0 +1,193 @@ +// Copyright (c) 2019 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 native + +import ( + "fmt" + "math" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/errors" + "github.com/m3db/m3/src/query/graphite/ts" +) + +// summarize summarizes each series into interval buckets of a certain size. +func summarize( + ctx *common.Context, + series singlePathSpec, + intervalS, fname string, + alignToFrom bool, +) (ts.SeriesList, error) { + if fname == "" { + fname = "sum" + } + + interval, err := common.ParseInterval(intervalS) + if err != nil || interval <= 0 { + err := errors.NewInvalidParamsError(fmt.Errorf( + "invalid interval %s: %v", interval, err)) + return ts.SeriesList{}, err + } + + f, fexists := summarizeFuncs[fname] + if !fexists { + err := errors.NewInvalidParamsError(fmt.Errorf( + "invalid func %s", fname)) + return ts.SeriesList{}, err + } + + alignString := "" + if alignToFrom { + alignString = ", true" + } + + results := make([]*ts.Series, len(series.Values)) + for i, series := range series.Values { + name := fmt.Sprintf("summarize(%s, \"%s\", \"%s\"%s)", series.Name(), intervalS, fname, alignString) + results[i] = summarizeTimeSeries(ctx, name, series, interval, f.consolidationFunc, alignToFrom) + } + + r := ts.SeriesList(series) + r.Values = results + return r, nil +} + +type summarizeBucket struct { + count int + accum float64 +} + +func summarizeTimeSeries( + ctx *common.Context, + newName string, + series *ts.Series, + interval time.Duration, + f ts.ConsolidationFunc, + alignToFrom bool, +) *ts.Series { + var ( + startTimeInSecs = int(series.StartTime().Unix()) + intervalInSecs = int(interval / time.Second) + intervalInMsecs = intervalInSecs * 1000 + buckets = make(map[int]*summarizeBucket) + ) + + for i := 0; i < series.Len(); i++ { + timestamp, n := int(series.StartTimeForStep(i).Unix()), series.ValueAt(i) + if math.IsNaN(n) { + continue + } + + bucketInterval := timestamp - (timestamp % intervalInSecs) + if alignToFrom { + bucketInterval = (timestamp - startTimeInSecs) / intervalInSecs + } + + if bucket, exists := buckets[bucketInterval]; exists { + bucket.accum = f(bucket.accum, n, bucket.count) + bucket.count++ + } else { + buckets[bucketInterval] = &summarizeBucket{1, n} + } + } + + var ( + newStart = series.StartTime() + newEnd = series.EndTime() + ) + + if !alignToFrom { + newStartInSecs, newEndInSecs := newStart.Unix(), newEnd.Unix() + newStart = time.Unix(newStartInSecs-newStartInSecs%int64(intervalInSecs), 0) + newEnd = time.Unix(newEndInSecs-newEndInSecs%int64(intervalInSecs)+int64(intervalInSecs), 0) + } + + var ( + numSteps = ts.NumSteps(newStart, newEnd, intervalInMsecs) + newValues = ts.NewValues(ctx, intervalInMsecs, numSteps) + ) + + for timestamp, i := newStart, 0; i < newValues.Len(); timestamp, i = timestamp.Add(interval), i+1 { + timestampInSecs := int(timestamp.Unix()) + var bucketInterval int + if alignToFrom { + bucketInterval = (timestampInSecs - startTimeInSecs) / intervalInSecs + } else { + bucketInterval = timestampInSecs - (timestampInSecs % intervalInSecs) + } + + bucket, bucketExists := buckets[bucketInterval] + if bucketExists { + newValues.SetValueAt(i, bucket.accum) + } + } + return ts.NewSeries(ctx, newName, newStart, newValues) +} + +// specificationFunc determines the output series specification given a series list. +type specificationFunc func(ts.SeriesList) string + +func sumSpecificationFunc(series ts.SeriesList) string { + return wrapPathExpr("sumSeries", series) +} + +func averageSpecificationFunc(series ts.SeriesList) string { + return wrapPathExpr("averageSeries", series) +} + +func maxSpecificationFunc(series ts.SeriesList) string { + return wrapPathExpr("maxSeries", series) +} + +func minSpecificationFunc(series ts.SeriesList) string { + return wrapPathExpr("minSeries", series) +} + +func lastSpecificationFunc(series ts.SeriesList) string { + return wrapPathExpr("lastSeries", series) +} + +type funcInfo struct { + consolidationFunc ts.ConsolidationFunc + specificationFunc specificationFunc +} + +var ( + sumFuncInfo = funcInfo{ts.Sum, sumSpecificationFunc} + maxFuncInfo = funcInfo{ts.Max, maxSpecificationFunc} + minFuncInfo = funcInfo{ts.Min, minSpecificationFunc} + lastFuncInfo = funcInfo{ts.Last, lastSpecificationFunc} + avgFuncInfo = funcInfo{ts.Avg, averageSpecificationFunc} + + summarizeFuncs = map[string]funcInfo{ + "sum": sumFuncInfo, + "max": maxFuncInfo, + "min": minFuncInfo, + "last": lastFuncInfo, + "avg": avgFuncInfo, + "sumSeries": sumFuncInfo, + "maxSeries": maxFuncInfo, + "minSeries": minFuncInfo, + "averageSeries": avgFuncInfo, + "": sumFuncInfo, + } +) diff --git a/src/query/graphite/native/summarize_test.go b/src/query/graphite/native/summarize_test.go new file mode 100644 index 0000000000..40b0f1ecb3 --- /dev/null +++ b/src/query/graphite/native/summarize_test.go @@ -0,0 +1,134 @@ +// Copyright (c) 2019 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 native + +import ( + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/common" + "github.com/m3db/m3/src/query/graphite/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestSummarize(t *testing.T) { + // NB(mmihic): Intentionally not aligned on summarization boundaries + var ( + start = time.Unix(131, 0) + end = time.Unix(251, 0) + ctx = common.NewContext(common.ContextOptions{Start: start, End: end}) + vals = ts.NewValues(ctx, 10000, 12) + series = []*ts.Series{ts.NewSeries(ctx, "foo", start, vals)} + ) + + defer ctx.Close() + + // 0+1+2 + // 3+4+5 + // 6+7+8 + // 9+10+11 + for i := 0; i < vals.Len(); i++ { + vals.SetValueAt(i, float64(i)) + } + + tests := []struct { + name string + interval string + fname string + alignToFrom bool + expectedStart time.Time + expectedEnd time.Time + expectedVals []float64 + }{ + {"summarize(foo, \"30s\", \"sum\")", "30s", "", false, + time.Unix(120, 0), time.Unix(270, 0), + []float64{1, 9, 18, 27, 11}, + }, + {"summarize(foo, \"30s\", \"sum\", true)", "30s", "", true, + start, end, + []float64{3, 12, 21, 30}, + }, + {"summarize(foo, \"1min\", \"sum\")", "1min", "", false, + time.Unix(120, 0), time.Unix(300, 0), + []float64{10, 45, 11}, + }, + {"summarize(foo, \"1min\", \"sum\", true)", "1min", "", true, + start, end, + []float64{15, 51}, + }, + {"summarize(foo, \"1min\", \"avg\")", "1min", "avg", false, + time.Unix(120, 0), time.Unix(300, 0), + []float64{2, 7.5, 11}, + }, + {"summarize(foo, \"1min\", \"last\")", "1min", "last", false, + start.Truncate(time.Minute), end.Truncate(time.Minute).Add(time.Minute), + []float64{4, 10, 11}, + }, + } + + for _, test := range tests { + outSeries, err := summarize(ctx, singlePathSpec{ + Values: series, + }, test.interval, test.fname, test.alignToFrom) + require.NoError(t, err) + require.Equal(t, 1, len(outSeries.Values)) + + out := outSeries.Values[0] + assert.Equal(t, test.name, out.Name(), "incorrect name for %s", test.name) + assert.Equal(t, test.expectedStart, out.StartTime(), "incorrect start for %s", test.name) + assert.Equal(t, test.expectedEnd, out.EndTime(), "incorrect end for %s", test.name) + require.Equal(t, len(test.expectedVals), out.Len(), "incorrect len for %s", test.name) + + for i := 0; i < out.Len(); i++ { + assert.Equal(t, test.expectedVals[i], out.ValueAt(i), "incorrect val %d for %s", i, test.name) + } + } + + _, err := summarize(ctx, singlePathSpec{ + Values: series, + }, "0min", "avg", false) + require.Error(t, err) +} + +func TestSummarizeInvalidInterval(t *testing.T) { + + var ( + start = time.Unix(131, 0) + end = time.Unix(251, 0) + ctx = common.NewContext(common.ContextOptions{Start: start, End: end}) + vals = ts.NewValues(ctx, 10000, 12) + series = []*ts.Series{ts.NewSeries(ctx, "foo", start, vals)} + ) + + defer ctx.Close() + + _, err := summarize(ctx, singlePathSpec{ + Values: series, + }, "0min", "avg", false) + require.Error(t, err) + + _, err = summarize(ctx, singlePathSpec{ + Values: series, + }, "-1hour", "avg", false) + require.Error(t, err) +} diff --git a/src/query/graphite/stats/statistics.go b/src/query/graphite/stats/statistics.go new file mode 100644 index 0000000000..156fc80e3e --- /dev/null +++ b/src/query/graphite/stats/statistics.go @@ -0,0 +1,200 @@ +// Copyright (c) 2019 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 stats + +import ( + "math" +) + +// Values presents a set of data values as an array, for the purposes of aggregation +type Values interface { + // Len returns the number of values present + Len() int + + // ValueAt returns the value at the nth element + ValueAt(n int) float64 +} + +// MutableValues is a set of data values that can be modified +type MutableValues interface { + Values + + // SetValueAt sets the value at the nth element + SetValueAt(n int, v float64) +} + +// Float64Values is a simple Values implementation around a slice +type Float64Values []float64 + +// Len returns the number of elements in the array +func (vals Float64Values) Len() int { return len(vals) } + +// ValueAt returns the value at the nth element +func (vals Float64Values) ValueAt(n int) float64 { return vals[n] } + +// SetValueAt sets the value at the nth element +func (vals Float64Values) SetValueAt(n int, v float64) { vals[n] = v } + +// Statistics are the computation of standard statistics (min, max, mean, count, stddev) +// over a group of values. +type Statistics struct { + Min float64 + Max float64 + Mean float64 + Count uint + Sum float64 + StdDev float64 +} + +// Merge merges a group of statistics +func Merge(statistics []Statistics) Statistics { + var ( + count uint + min, max, mean, sum float64 + ) + + for _, a := range statistics { + if a.Count == 0 { + continue + } + + if count == 0 { + min, max = a.Min, a.Max + } else { + min, max = math.Min(min, a.Min), math.Max(max, a.Max) + } + + priorCount := count + count += a.Count + sum += a.Sum + mean = ((a.Mean * float64(a.Count)) + (mean * float64(priorCount))) / float64(count) + } + + if count == 0 { + return Statistics{} + } + + var sum1, sum2 float64 + for _, a := range statistics { + if a.Count == 0 { + continue + } + + variance := a.StdDev * a.StdDev + avg := a.Mean + sum1 += float64(a.Count) * variance + sum2 += float64(a.Count) * math.Pow(avg-mean, 2) + } + + variance := ((sum1 + sum2) / float64(count)) + return Statistics{ + Count: count, + Min: min, + Max: max, + Mean: mean, + Sum: sum, + StdDev: math.Sqrt(variance), + } +} + +func calc(values Values) (uint, float64, float64, float64, float64, float64) { + count := uint(0) + sum := float64(0) + min := math.MaxFloat64 + max := -math.MaxFloat64 + for i := 0; i < values.Len(); i++ { + n := values.ValueAt(i) + if math.IsNaN(n) { + continue + } + count++ + sum += n + min = math.Min(n, min) + max = math.Max(n, max) + } + + if count == 0 { + nan := math.NaN() + return 0, nan, nan, nan, nan, nan + } + + mean := float64(0) + if count > 0 { + mean = sum / float64(count) + } + + stddev := float64(0) + if count > 1 { + m2 := float64(0) + for i := 0; i < values.Len(); i++ { + n := values.ValueAt(i) + if math.IsNaN(n) { + continue + } + + diff := n - mean + m2 += diff * diff + } + + variance := m2 / float64(count-1) + stddev = math.Sqrt(variance) + } + return count, min, max, mean, sum, stddev +} + +// Calc calculates statistics for a set of values +func Calc(values Values) Statistics { + count, min, max, mean, sum, stddev := calc(values) + return Statistics{ + Count: count, + Min: min, + Max: max, + Mean: mean, + Sum: sum, + StdDev: stddev, + } +} + +// SingleCountStatistics returns Statistics for a single value +func SingleCountStatistics(value float64) Statistics { + return Statistics{ + Count: 1, + Min: value, + Max: value, + Sum: value, + Mean: value, + StdDev: 0, + } +} + +// ZeroCountStatistics returns statistics when no values are present +// (or when all values are NaNs) +func ZeroCountStatistics() Statistics { + nan := math.NaN() + return Statistics{ + Count: 0, + Min: nan, + Max: nan, + Sum: nan, + Mean: nan, + StdDev: nan, + } +} diff --git a/src/query/graphite/stats/statistics_test.go b/src/query/graphite/stats/statistics_test.go new file mode 100644 index 0000000000..bad60cb339 --- /dev/null +++ b/src/query/graphite/stats/statistics_test.go @@ -0,0 +1,67 @@ +// Copyright (c) 2019 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 stats + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestCalc(t *testing.T) { + vals := Float64Values{20.5, 13, 74, 98, 12, 76.4, 264, 201, 98, 30, 16, 12} + a := Calc(vals) + assert.Equal(t, Statistics{ + Min: 12, + Max: 264, + Mean: 76.24166666666666, + Count: 12, + Sum: 914.9, + StdDev: 81.4067839292093, + }, a) +} + +func TestMergeAggregations(t *testing.T) { + statistics := Merge([]Statistics{ + Calc(Float64Values{20.5, 13, 74, 98}), + Calc(Float64Values{12, 76.4, 264, 201}), + Calc(Float64Values{98, 30, 16, 12}), + Statistics{ + Count: 0, + }, + }) + assert.Equal(t, Statistics{ + Min: 12, + Max: 264, + Mean: 76.24166666666666, + Count: 12, + Sum: 914.9, + StdDev: 86.30370872036085, + }, statistics) +} + +func TestMergeEmptyAggregations(t *testing.T) { + statistics := Merge([]Statistics{ + Statistics{Count: 0}, + Statistics{Count: 0}, + }) + assert.Equal(t, Statistics{}, statistics) +} diff --git a/src/query/graphite/storage/converter.go b/src/query/graphite/storage/converter.go new file mode 100644 index 0000000000..7ea9219310 --- /dev/null +++ b/src/query/graphite/storage/converter.go @@ -0,0 +1,101 @@ +// Copyright (c) 2019 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 storage + +import ( + "bytes" + "fmt" + "strings" + + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/models" +) + +const ( + carbonSeparatorByte = byte('.') + carbonGlobRune = '*' +) + +func glob(metric string) []byte { + globLen := len(metric) + for _, c := range metric { + if c == carbonGlobRune { + globLen++ + } + } + + glob := make([]byte, globLen) + i := 0 + for _, c := range metric { + if c == carbonGlobRune { + glob[i] = carbonSeparatorByte + i++ + } + + glob[i] = byte(c) + i++ + } + + return glob +} + +func convertMetricPartToMatcher(count int, metric string) models.Matcher { + return models.Matcher{ + Type: models.MatchRegexp, + Name: graphite.TagName(count), + Value: glob(metric), + } +} + +func matcherTerminator(count int) models.Matcher { + return models.Matcher{ + Type: models.MatchNotRegexp, + Name: graphite.TagName(count), + Value: []byte(".*"), + } +} + +func convertTagsToMetricName(tags models.Tags) (string, error) { + var builder strings.Builder + for i, tag := range tags.Tags { + if bytes.Compare(tag.Name, graphite.TagName(i)) != 0 { + // If not in order or a completely different named tag + // then abort, we can't generate the metric name + err := fmt.Errorf("unexpected tag name: expected=%s, actual=%s", + graphite.TagName(i), tag.Name) + return "", err + } + + _, err := builder.Write(tag.Value) + if err != nil { + return "", err + } + + if i != len(tags.Tags)-1 { + _, err := builder.WriteRune('.') + if err != nil { + return "", err + } + } + } + + return builder.String(), nil +} diff --git a/src/query/graphite/storage/converter_test.go b/src/query/graphite/storage/converter_test.go new file mode 100644 index 0000000000..dd25dec31f --- /dev/null +++ b/src/query/graphite/storage/converter_test.go @@ -0,0 +1,75 @@ +// Copyright (c) 2019 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 storage + +import ( + "testing" + + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/models" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestGlob(t *testing.T) { + noGlob := "some_sort of-string,with~no=globbing" + expected := []byte(noGlob) + actual := glob(noGlob) + require.Equal(t, actual, expected) + + globbed := "foo*bar" + expected = []byte("foo.*bar") + actual = glob(globbed) + require.Equal(t, actual, expected) + + globAndRegex := "foo*bar[rz]*(qux|quail)" + expected = []byte("foo.*bar[rz].*(qux|quail)") + actual = glob(globAndRegex) + require.Equal(t, actual, expected) +} + +func TestConvertMetricPartToMatcher(t *testing.T) { + for i := 0; i < 100; i++ { + globAndRegex := "foo*bar[rz]*(qux|quail)" + expected := models.Matcher{ + Type: models.MatchRegexp, + Name: graphite.TagName(i), + Value: []byte("foo.*bar[rz].*(qux|quail)"), + } + + actual := convertMetricPartToMatcher(i, globAndRegex) + assert.Equal(t, expected, actual) + } +} + +func TestGetMatcherTerminator(t *testing.T) { + for i := 0; i < 100; i++ { + expected := models.Matcher{ + Type: models.MatchNotRegexp, + Name: graphite.TagName(i), + Value: []byte(".*"), + } + + actual := matcherTerminator(i) + assert.Equal(t, expected, actual) + } +} diff --git a/src/query/graphite/storage/m3_wrapper.go b/src/query/graphite/storage/m3_wrapper.go new file mode 100644 index 0000000000..c081e50d2e --- /dev/null +++ b/src/query/graphite/storage/m3_wrapper.go @@ -0,0 +1,156 @@ +// Copyright (c) 2019 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 storage + +import ( + "context" + "errors" + "time" + + xctx "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/graphite/ts" + "github.com/m3db/m3/src/query/models" + "github.com/m3db/m3/src/query/storage" + m3ts "github.com/m3db/m3/src/query/ts" +) + +var ( + errSeriesNoResolution = errors.New("series has no resolution set") +) + +type m3WrappedStore struct { + m3 storage.Storage +} + +// NewM3WrappedStorage creates a graphite storage wrapper around an m3query +// storage instance. +func NewM3WrappedStorage(m3storage storage.Storage) Storage { + return &m3WrappedStore{m3: m3storage} +} + +// TranslateQueryToMatchers converts a graphite query to tag matcher pairs. +func TranslateQueryToMatchers(query string) models.Matchers { + metricLength := graphite.CountMetricParts(query) + matchers := make(models.Matchers, metricLength) + for i := 0; i < metricLength; i++ { + metric := graphite.ExtractNthMetricPart(query, i) + if len(metric) > 0 { + matchers[i] = convertMetricPartToMatcher(i, metric) + } + } + + return matchers +} + +// GetQueryTerminatorTagName will return the name for the terminator matcher in +// the given pattern. This is useful for filtering out any additional results. +func GetQueryTerminatorTagName(query string) []byte { + metricLength := graphite.CountMetricParts(query) + return graphite.TagName(metricLength) +} + +func translateQuery(query string, opts FetchOptions) *storage.FetchQuery { + metricLength := graphite.CountMetricParts(query) + matchers := make(models.Matchers, metricLength+1) + for i := 0; i < metricLength; i++ { + metric := graphite.ExtractNthMetricPart(query, i) + if len(metric) > 0 { + matchers[i] = convertMetricPartToMatcher(i, metric) + } + } + + // Add a terminator matcher at the end to ensure expansion is terminated at + // the last given metric part. + matchers[metricLength] = matcherTerminator(metricLength) + return &storage.FetchQuery{ + Raw: query, + TagMatchers: matchers, + Start: opts.StartTime, + End: opts.EndTime, + // NB: interval is not used for initial consolidation step from the storage + // so it's fine to use default here. + Interval: time.Duration(0), + } +} + +func translateTimeseries( + ctx xctx.Context, + m3list m3ts.SeriesList, + start, end time.Time, +) ([]*ts.Series, error) { + series := make([]*ts.Series, len(m3list)) + for i, m3series := range m3list { + resolution := m3series.Resolution() + if resolution <= 0 { + return nil, errSeriesNoResolution + } + + length := int(end.Sub(start) / resolution) + millisPerStep := int(resolution / time.Millisecond) + values := ts.NewValues(ctx, millisPerStep, length) + for _, datapoint := range m3series.Values().Datapoints() { + index := int(datapoint.Timestamp.Sub(start) / resolution) + if index < 0 || index >= length { + // Outside of range requested + continue + } + values.SetValueAt(index, datapoint.Value) + } + + name := m3series.Name() + if tags := m3series.Tags; tags.Len() > 0 { + // Need to flatten the name back into graphite format + newName, err := convertTagsToMetricName(tags) + if err != nil { + return nil, err + } + + name = newName + } + + series[i] = ts.NewSeries(ctx, name, start, values) + } + + return series, nil +} + +func (s *m3WrappedStore) FetchByQuery( + ctx xctx.Context, query string, opts FetchOptions, +) (*FetchResult, error) { + m3query := translateQuery(query, opts) + m3ctx, cancel := context.WithTimeout(ctx.RequestContext(), opts.Timeout) + defer cancel() + + m3result, err := s.m3.Fetch(m3ctx, m3query, + storage.NewFetchOptions()) + if err != nil { + return nil, err + } + + series, err := translateTimeseries(ctx, m3result.SeriesList, + opts.StartTime, opts.EndTime) + if err != nil { + return nil, err + } + + return NewFetchResult(ctx, series), nil +} diff --git a/src/query/graphite/storage/m3_wrapper_test.go b/src/query/graphite/storage/m3_wrapper_test.go new file mode 100644 index 0000000000..a84d8f899e --- /dev/null +++ b/src/query/graphite/storage/m3_wrapper_test.go @@ -0,0 +1,157 @@ +// Copyright (c) 2019 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 storage + +import ( + "context" + "fmt" + "testing" + "time" + + xctx "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/query/models" + "github.com/m3db/m3/src/query/storage" + "github.com/m3db/m3/src/query/storage/mock" + m3ts "github.com/m3db/m3/src/query/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestTranslateQuery(t *testing.T) { + query := `foo.ba[rz].q*x.terminator.will.be.back?` + end := time.Now() + start := end.Add(time.Hour * -2) + opts := FetchOptions{ + StartTime: start, + EndTime: end, + DataOptions: DataOptions{ + Timeout: time.Minute, + }, + } + + translated := translateQuery(query, opts) + assert.Equal(t, end, translated.End) + assert.Equal(t, start, translated.Start) + assert.Equal(t, time.Duration(0), translated.Interval) + assert.Equal(t, query, translated.Raw) + matchers := translated.TagMatchers + expected := models.Matchers{ + {Type: models.MatchRegexp, Name: graphite.TagName(0), Value: []byte("foo")}, + {Type: models.MatchRegexp, Name: graphite.TagName(1), Value: []byte("ba[rz]")}, + {Type: models.MatchRegexp, Name: graphite.TagName(2), Value: []byte("q.*x")}, + {Type: models.MatchRegexp, Name: graphite.TagName(3), Value: []byte("terminator")}, + {Type: models.MatchRegexp, Name: graphite.TagName(4), Value: []byte("will")}, + {Type: models.MatchRegexp, Name: graphite.TagName(5), Value: []byte("be")}, + {Type: models.MatchRegexp, Name: graphite.TagName(6), Value: []byte("back?")}, + {Type: models.MatchNotRegexp, Name: graphite.TagName(7), Value: []byte(".*")}, + } + + assert.Equal(t, expected, matchers) +} + +func TestTranslateTimeseries(t *testing.T) { + ctx := xctx.New() + resolution := 10 * time.Second + steps := 1 + start := time.Now() + end := start.Add(time.Duration(steps) * resolution) + expected := 5 + seriesList := make(m3ts.SeriesList, expected) + for i := 0; i < expected; i++ { + vals := m3ts.NewFixedStepValues(resolution, steps, float64(i), start) + series := m3ts.NewSeries(fmt.Sprint("a", i), vals, models.NewTags(0, nil)) + series.SetResolution(resolution) + seriesList[i] = series + } + + translated, err := translateTimeseries(ctx, seriesList, start, end) + require.NoError(t, err) + + require.Equal(t, expected, len(translated)) + for i, tt := range translated { + ex := []float64{float64(i)} + assert.Equal(t, ex, tt.SafeValues()) + assert.Equal(t, fmt.Sprint("a", i), tt.Name()) + } +} + +func TestTranslateTimeseriesWithTags(t *testing.T) { + ctx := xctx.New() + resolution := 10 * time.Second + steps := 1 + start := time.Now() + end := start.Add(time.Duration(steps) * resolution) + expected := 5 + seriesList := make(m3ts.SeriesList, expected) + for i := 0; i < expected; i++ { + vals := m3ts.NewFixedStepValues(resolution, steps, float64(i), start) + series := m3ts.NewSeries(fmt.Sprint("a", i), vals, models.NewTags(0, nil)) + series.SetResolution(resolution) + seriesList[i] = series + } + + translated, err := translateTimeseries(ctx, seriesList, start, end) + require.NoError(t, err) + + require.Equal(t, expected, len(translated)) + for i, tt := range translated { + ex := []float64{float64(i)} + assert.Equal(t, ex, tt.SafeValues()) + assert.Equal(t, fmt.Sprint("a", i), tt.Name()) + } +} + +func TestFetchByQuery(t *testing.T) { + store := mock.NewMockStorage() + start := time.Now().Add(time.Hour * -1) + resolution := 10 * time.Second + steps := 3 + vals := m3ts.NewFixedStepValues(resolution, steps, 3, start) + seriesList := m3ts.SeriesList{ + m3ts.NewSeries("a", vals, models.NewTags(0, nil)), + } + for _, series := range seriesList { + series.SetResolution(resolution) + } + + store.SetFetchResult(&storage.FetchResult{SeriesList: seriesList}, nil) + wrapper := NewM3WrappedStorage(store) + ctx := xctx.New() + ctx.SetRequestContext(context.TODO()) + end := time.Now() + opts := FetchOptions{ + StartTime: start, + EndTime: end, + DataOptions: DataOptions{ + Timeout: time.Minute, + }, + } + + query := "a*b" + result, err := wrapper.FetchByQuery(ctx, query, opts) + assert.NoError(t, err) + require.Equal(t, 1, len(result.SeriesList)) + series := result.SeriesList[0] + assert.Equal(t, "a", series.Name()) + assert.Equal(t, []float64{3, 3, 3}, series.SafeValues()) +} diff --git a/src/query/graphite/storage/storage.go b/src/query/graphite/storage/storage.go new file mode 100644 index 0000000000..2047eed027 --- /dev/null +++ b/src/query/graphite/storage/storage.go @@ -0,0 +1,85 @@ +// Copyright (c) 2019 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 storage + +import ( + "sync" + "time" + + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/ts" +) + +// FetchOptions provide context to a fetch +type FetchOptions struct { + StartTime time.Time // The start time for the fetch + EndTime time.Time // The end time for the fetch + DataOptions +} + +// DataOptions provide data context +type DataOptions struct { + Timeout time.Duration // Whether to use a custom timeout, zero if no or positive if yes +} + +// Storage provides an interface for retrieving timeseries values or names based upon +// a query or path +type Storage interface { + // FetchByQuery fetches timeseries data based on a query + FetchByQuery( + ctx context.Context, query string, opts FetchOptions, + ) (*FetchResult, error) +} + +// FetchResult provides a fetch result and meta information +type FetchResult struct { + SeriesList []*ts.Series // The aggregated list of results across all underlying storage calls +} + +// Close will return the fetch result to the pool. +func (fr *FetchResult) Close() error { + fr.SeriesList = nil + fetchResultPool.Put(fr) + return nil +} + +// Reset will wipe out existing fetch result data. +func (fr *FetchResult) Reset() { + fr.SeriesList = nil +} + +var ( + fetchResultPool = &sync.Pool{ + New: func() interface{} { + return &FetchResult{} + }, + } +) + +// NewFetchResult is a convenience method for creating a FetchResult +func NewFetchResult(ctx context.Context, seriesList []*ts.Series) *FetchResult { + fetchResult := fetchResultPool.Get().(*FetchResult) + fetchResult.Reset() + + fetchResult.SeriesList = seriesList + ctx.RegisterCloser(fetchResult) + return fetchResult +} diff --git a/src/query/graphite/testing/test_utils.go b/src/query/graphite/testing/test_utils.go new file mode 100644 index 0000000000..3b08df0e09 --- /dev/null +++ b/src/query/graphite/testing/test_utils.go @@ -0,0 +1,146 @@ +// Copyright (c) 2019 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 testing + +import ( + "fmt" + "math" + + "github.com/stretchr/testify/assert" +) + +func toFloat(x interface{}) (float64, bool) { + var xf float64 + xok := true + + switch xn := x.(type) { + case uint8: + xf = float64(xn) + case uint16: + xf = float64(xn) + case uint32: + xf = float64(xn) + case uint64: + xf = float64(xn) + case int: + xf = float64(xn) + case int8: + xf = float64(xn) + case int16: + xf = float64(xn) + case int32: + xf = float64(xn) + case int64: + xf = float64(xn) + case float32: + xf = float64(xn) + case float64: + xf = xn + default: + xok = false + } + + return xf, xok +} + +func castToFloats( + t assert.TestingT, + expected, actual interface{}, +) (float64, float64, bool) { + af, aok := toFloat(expected) + bf, bok := toFloat(actual) + + if !aok || !bok { + return 0, 0, assert.Fail(t, "expected or actual are unexpected types") + } + + return af, bf, true +} + +// EqualWithNaNs compares two numbers for equality, accounting for NaNs. +func EqualWithNaNs( + t assert.TestingT, + expected, actual interface{}, + msgAndArgs ...interface{}, +) bool { + af, bf, ok := castToFloats(t, expected, actual) + if !ok { + return ok + } + + if math.IsNaN(af) && math.IsNaN(bf) { + return true + } + + return assert.Equal(t, af, bf, msgAndArgs) +} + +// InDeltaWithNaNs compares two floats for equality within a delta, +// accounting for NaNs. +func InDeltaWithNaNs( + t assert.TestingT, + expected, actual interface{}, + delta float64, + msgAndArgs ...interface{}, +) bool { + af, bf, ok := castToFloats(t, expected, actual) + if !ok { + return ok + } + + if math.IsNaN(af) && math.IsNaN(bf) { + return true + } + + dt := af - bf + if dt < -delta || dt > delta { + return assert.Fail(t, + fmt.Sprintf( + "Max difference between %v and %v allowed is %v, but difference was %v", + expected, actual, delta, dt), msgAndArgs...) + } + + return true +} + +// Equalish asserts that two objects are equal. Looser than assert.Equal since +// it checks for equality of printing expected vs printing actual. +// +// assert.Equal(t, 123, 123, "123 and 123 should be equal") +// +// Returns whether the assertion was successful (true) or not (false). +func Equalish( + t assert.TestingT, + expected, actual interface{}, + msgAndArgs ...interface{}, +) bool { + if assert.ObjectsAreEqual(expected, actual) { + return true + } + + // Last ditch effort + if fmt.Sprintf("%#v", expected) == fmt.Sprintf("%#v", actual) { + return true + } + + return assert.Fail(t, fmt.Sprintf("Not equal: %#v (expected)\n"+ + " != %#v (actual)", expected, actual), msgAndArgs...) +} diff --git a/src/query/graphite/ts/consolidation_approach.go b/src/query/graphite/ts/consolidation_approach.go new file mode 100644 index 0000000000..c12a453664 --- /dev/null +++ b/src/query/graphite/ts/consolidation_approach.go @@ -0,0 +1,64 @@ +// Copyright (c) 2019 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 ts + +import ( + "fmt" +) + +// ConsolidationApproach defines an approach to consolidating multiple datapoints +type ConsolidationApproach string + +// The standard set of consolidation functions +const ( + ConsolidationAvg ConsolidationApproach = "avg" + ConsolidationMin ConsolidationApproach = "min" + ConsolidationMax ConsolidationApproach = "max" + ConsolidationSum ConsolidationApproach = "sum" + ConsolidationAverage ConsolidationApproach = "average" // just an alias to avg but for backward-compatibility +) + +// SafeFunc returns a boolean indicating whether this is a valid consolidation approach, +// and if so, the corresponding ConsolidationFunc. +func (ca ConsolidationApproach) SafeFunc() (ConsolidationFunc, bool) { + f, ok := consolidationFuncs[ca] + return f, ok +} + +// Func returns the ConsolidationFunc implementing the ConsolidationApproach +func (ca ConsolidationApproach) Func() ConsolidationFunc { + f, ok := ca.SafeFunc() + if !ok { + panic(fmt.Sprintf("No consolidation func for %s", ca)) + } + + return f +} + +var ( + consolidationFuncs = map[ConsolidationApproach]ConsolidationFunc{ + ConsolidationAvg: Avg, + ConsolidationAverage: Avg, + ConsolidationMin: Min, + ConsolidationMax: Max, + ConsolidationSum: Sum, + } +) diff --git a/src/query/graphite/ts/datapoint.go b/src/query/graphite/ts/datapoint.go new file mode 100644 index 0000000000..cd827bdd03 --- /dev/null +++ b/src/query/graphite/ts/datapoint.go @@ -0,0 +1,96 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + "time" + + "github.com/m3db/m3/src/query/graphite/stats" +) + +// A Datapoint is a single data value reported at a given time +type Datapoint struct { + Timestamp time.Time + Value float64 +} + +// ValueIsNaN returns true iff underlying value is NaN +func (d Datapoint) ValueIsNaN() bool { return math.IsNaN(d.Value) } + +// DatapointsByTimestamp is a sortable interface for datapoints +type DatapointsByTimestamp []Datapoint + +// Len is the length of the datapoints +func (p DatapointsByTimestamp) Len() int { return len(p) } + +// Less compares two datapoints by timestamp +func (p DatapointsByTimestamp) Less(i, j int) bool { return p[i].Timestamp.Before(p[j].Timestamp) } + +// Swap swaps two datapoints +func (p DatapointsByTimestamp) Swap(i, j int) { p[i], p[j] = p[j], p[i] } + +// ConsolidatedValue represents a time window of consolidated data +type ConsolidatedValue struct { + // StartTime is the start time of the time window covered by this + // consolidation + StartTime time.Time + + // EndTime is the end time of the time window covered by this + // consolidation + EndTime time.Time + + // Values is the statistics for that consolidated time window + Values stats.Statistics +} + +// ConsolidatedValuesByStartTime is a sortable interface for consolidated values +type ConsolidatedValuesByStartTime []ConsolidatedValue + +// Len is the length of the values +func (p ConsolidatedValuesByStartTime) Len() int { return len(p) } + +// Less compares two values by start time +func (p ConsolidatedValuesByStartTime) Less(i, j int) bool { + return p[i].StartTime.Before(p[j].StartTime) +} + +// Swap swaps two values +func (p ConsolidatedValuesByStartTime) Swap(i, j int) { p[i], p[j] = p[j], p[i] } + +// Datapoints is a list of datapoints that implement the stats.Values interface. +type Datapoints []Datapoint + +// Len is the length of the array. +func (d Datapoints) Len() int { return len(d) } + +// ValueAt returns the value at the nth element. +func (d Datapoints) ValueAt(n int) float64 { return d[n].Value } + +// AllNaN returns true if all the values are NaN +func (d Datapoints) AllNaN() bool { + for _, dp := range d { + if !dp.ValueIsNaN() { + return false + } + } + return true +} diff --git a/src/query/graphite/ts/lttb.go b/src/query/graphite/ts/lttb.go new file mode 100644 index 0000000000..08eabdf18c --- /dev/null +++ b/src/query/graphite/ts/lttb.go @@ -0,0 +1,179 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + "time" +) + +// LTTB down-samples the data to contain only threshold number of points that +// have the same visual shape as the original data. Inspired from +// https://github.com/dgryski/go-lttb which is based on +// https://skemman.is/bitstream/1946/15343/3/SS_MSthesis.pdf +func LTTB(b *Series, start time.Time, end time.Time, millisPerStep int) *Series { + if end.After(b.EndTime()) { + end = b.EndTime() + } + + seriesValuesPerStep := millisPerStep / b.MillisPerStep() + seriesStart, seriesEnd := b.StepAtTime(start), b.StepAtTime(end) + + // This threshold is different than max datapoints since we ensure step size is an integer multiple of original series step + threshold := int(math.Ceil(float64(seriesEnd-seriesStart) / float64(seriesValuesPerStep))) + if threshold == 0 || threshold > b.Len() { + return b // Nothing to do + } + + values := NewValues(b.ctx, millisPerStep, threshold) + // Bucket size. Leave room for start and end data points + every := float64(seriesValuesPerStep) + // Always add the first point + values.SetValueAt(0, b.ValueAt(seriesStart)) + // Set a to be the first chosen point + a := seriesStart + + bucketStart := seriesStart + 1 + bucketCenter := bucketStart + int(math.Floor(every)) + 1 + + for i := 0; i < threshold-2; i++ { + bucketEnd := bucketCenter + int(math.Floor(every)) + + // Calculate point average for next bucket (containing c) + avgRangeStart := bucketCenter + avgRangeEnd := bucketEnd + + if avgRangeEnd >= seriesEnd { + avgRangeEnd = seriesEnd + } + + avgRangeLength := float64(avgRangeEnd - avgRangeStart) + + var avgX, avgY float64 + var valuesRead int + for ; avgRangeStart < avgRangeEnd; avgRangeStart++ { + yVal := b.ValueAt(avgRangeStart) + if math.IsNaN(yVal) { + continue + } + valuesRead++ + avgX += float64(avgRangeStart) + avgY += yVal + } + + if valuesRead > 0 { + avgX /= avgRangeLength + avgY /= avgRangeLength + } else { + // If all nulls then should not assign a value to average + avgX = math.NaN() + avgY = math.NaN() + } + + // Get the range for this bucket + rangeOffs := bucketStart + rangeTo := bucketCenter + + // Point a + pointAX := float64(a) + pointAY := b.ValueAt(a) + + var nextA int + + // If all points in left or right bucket are null, then fallback to average + if math.IsNaN(avgY) || math.IsNaN(pointAY) { + nextA = indexClosestToAverage(b, rangeOffs, rangeTo) + } else { + nextA = indexWithLargestTriangle(b, rangeOffs, rangeTo, pointAX, pointAY, avgX, avgY) + } + + values.SetValueAt(i+1, b.ValueAt(nextA)) // Pick this point from the bucket + a = nextA // This a is the next a (chosen b) + + bucketStart = bucketCenter + bucketCenter = bucketEnd + } + + if values.Len() > 1 { + // Always add last if not just a single step + values.SetValueAt(values.Len()-1, b.ValueAt(seriesEnd-1)) + } + + // Derive a new series + sampledSeries := b.DerivedSeries(start, values) + return sampledSeries +} + +func indexWithLargestTriangle(b *Series, start int, end int, leftX float64, leftY float64, rightX float64, rightY float64) int { + // The original algorithm implementation initializes the maxArea as 0 which is a bug! + maxArea := -1.0 + var largestIndex int + + xDifference := leftX - rightX + yDifference := rightY - leftY + for index := start; index < end; index++ { + // Calculate triangle area over three buckets + area := xDifference*(b.ValueAt(index)-leftY) - (leftX-float64(index))*yDifference + // We only care about the relative area here. + area = math.Abs(area) + // Handle nulls properly + if math.IsNaN(area) { + area = 0 + } + + if area > maxArea { + maxArea = area + largestIndex = index + } + } + + return largestIndex +} + +func indexClosestToAverage(b *Series, start int, end int) int { + var sum float64 + var count int + for index := start; index < end; index++ { + if math.IsNaN(b.ValueAt(index)) { + continue + } + + sum += b.ValueAt(index) + count++ + } + + if count == 0 { + return start + } + + average := sum / float64(count) + minDifference := math.MaxFloat64 + closestIndex := start + for index := start; index < end; index++ { + difference := math.Abs(average - b.ValueAt(index)) + if !math.IsNaN(b.ValueAt(index)) && difference < minDifference { + closestIndex = index + minDifference = difference + } + } + + return closestIndex +} diff --git a/src/query/graphite/ts/lttb_test.go b/src/query/graphite/ts/lttb_test.go new file mode 100644 index 0000000000..a8e2a65dc5 --- /dev/null +++ b/src/query/graphite/ts/lttb_test.go @@ -0,0 +1,695 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/context" + + "github.com/stretchr/testify/require" +) + +func TestLTTB(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + seriesStart := time.Now() + stepSize := 1000 + values := NewValues(ctx, stepSize, 10) + for i := 0; i < values.Len(); i++ { + values.SetValueAt(i, float64(i+1)) + } + series := NewSeries(ctx, "foo", seriesStart, values) + + tests := []struct { + startOffset time.Duration + endOffset time.Duration + threshold int + newStep int + resultThreshold int + }{ + { + startOffset: 0, + endOffset: 0, + threshold: 3, + newStep: stepSize * 4, + }, + { + startOffset: 0, + endOffset: 0, + newStep: stepSize * 3, + threshold: 4, + }, + { + startOffset: 0, + endOffset: 0, + newStep: stepSize * 2, + threshold: 5, + }, + { + startOffset: 0, + endOffset: 0, + newStep: stepSize * 2, + threshold: 6, + resultThreshold: 5, + }, + { + startOffset: 0, + endOffset: 0, + newStep: stepSize * 2, + threshold: 7, + resultThreshold: 5, + }, + } + + for _, test := range tests { + start := series.StartTime().Add(test.startOffset) + end := series.EndTime().Add(test.endOffset) + threshold := test.threshold + millisPerStep := series.MillisPerStep() + // Calculate the new millisPerStep if the length exceeds the limit on number of data points + if (series.Len()) > threshold { + samplingMultiplier := math.Ceil(float64(series.Len()) / float64(threshold)) + millisPerStep = int(samplingMultiplier * float64(series.MillisPerStep())) + } + + result := LTTB(series, start, end, millisPerStep) + require.NotNil(t, result) + require.Equal(t, start, result.StartTime()) + require.False(t, end.After(result.EndTime())) + require.True(t, end.After(result.StartTimeForStep(result.Len()-1))) + + require.Equal(t, series.Specification, result.Specification) + require.Equal(t, series.Name(), result.Name()) + require.Equal(t, test.newStep, result.MillisPerStep()) + resultThreshold := test.threshold + if test.resultThreshold > 0 { + resultThreshold = test.resultThreshold + } + require.Equal(t, result.Len(), resultThreshold) + + } +} + +func TestLTTBWithNull(t *testing.T) { + ctx := context.New() + defer ctx.Close() + seriesStart := time.Now() + stepSize := 1000 + values := NewValues(ctx, stepSize, len(data)/50) + for i := 0; i < values.Len(); i++ { + values.SetValueAt(i, data[i].Y) + } + + values.SetValueAt(0, math.NaN()) + series := NewSeries(ctx, "foo", seriesStart, values) + + series = LTTB(series, series.StartTime(), series.EndTime(), 5000) + require.True(t, math.IsNaN(series.ValueAt(0))) + require.False(t, math.IsNaN(series.ValueAt(1))) +} + +func TestLTTBAverage(t *testing.T) { + ctx := context.New() + defer ctx.Close() + seriesStart := time.Now() + values := NewValues(ctx, 1000, 8) + values.SetValueAt(4, data[4].Y) + + series := NewSeries(ctx, "foo", seriesStart, values) + + series = LTTB(series, series.StartTime(), series.EndTime(), 3000) + require.False(t, math.IsNaN(series.ValueAt(1))) + require.InDelta(t, series.ValueAt(1), data[4].Y, 0.001) + + values = NewValues(ctx, 1000, 12) + values.SetValueAt(7, data[7].Y) + series = NewSeries(ctx, "foo", seriesStart, values) + series = LTTB(series, series.StartTime(), series.EndTime(), 3000) + require.False(t, math.IsNaN(series.ValueAt(2))) + require.InDelta(t, series.ValueAt(2), data[7].Y, 0.001) +} + +var data = []Point{ + {0, 29.357995947822218}, {1, 29.40932479606209}, {2, 29.28168582006162}, {3, 30.409965579108867}, {4, 30.7726859735917}, {5, 30.839942247539028}, {6, 30.760611642264667}, {7, 31.203663004229718}, {8, 31.38899603525572}, {9, 30.890299916955737}, + {10, 30.467811944911556}, {11, 30.596837868069542}, {12, 30.59789593509767}, {13, 30.19693062465079}, {14, 29.89081330734553}, {15, 29.54668002901058}, {16, 29.54890739422219}, {17, 30.53743760171474}, {18, 30.74066032317061}, {19, 30.3774450601516}, + {20, 30.095148889986568}, {21, 30.057979182917986}, {22, 30.364655421168525}, {23, 30.293450053773604}, {24, 30.14578230340987}, {25, 30.277772879951996}, {26, 30.3711931235659}, {27, 30.355932660992572}, {28, 29.994740831603046}, {29, 29.938012885023657}, + {30, 29.557268760451187}, {31, 29.16161297604625}, {32, 29.6921163421055}, {33, 30.246270628292226}, {34, 30.210955154680928}, {35, 29.382964668934058}, {36, 29.52892023906641}, {37, 29.719742065732202}, {38, 30.013856636945924}, {39, 29.818636169776926}, + {40, 27.931224826933345}, {41, 28.103057873678374}, {42, 28.276025902356782}, {43, 28.193497487789774}, {44, 28.22099171488288}, {45, 28.06872431241809}, {46, 27.239835885250365}, {47, 26.606504358317906}, {48, 27.220308136213916}, {49, 25.80790898978574}, + {50, 26.060334838062595}, {51, 25.89611036822564}, {52, 25.824433242208308}, {53, 25.89212193260212}, {54, 27.28963669872412}, {55, 27.208259984780806}, {56, 27.298211480886604}, {57, 27.056163307023077}, {58, 26.434701869643924}, {59, 26.21938188318228}, + {60, 26.402877759468772}, {61, 26.326019357967294}, {62, 27.592635104460253}, {63, 27.597468405434316}, {64, 27.533410081050647}, {65, 27.30713964830731}, {66, 27.170789712751404}, {67, 27.610166301145746}, {68, 27.619847332319885}, {69, 27.55974343996647}, + {70, 28.168803492093716}, {71, 28.284229862452584}, {72, 28.292638086760352}, {73, 28.523718178262335}, {74, 28.52268591410558}, {75, 28.576637206595993}, {76, 27.763715461946813}, {77, 27.48488075310065}, {78, 27.304006460281794}, {79, 26.902025619666333}, + {80, 26.690162079132378}, {81, 26.52277286738866}, {82, 26.09406789919051}, {83, 26.029039984983562}, {84, 26.145181108429618}, {85, 24.75335912549342}, {86, 24.787772395126176}, {87, 24.966652093726385}, {88, 25.33621421606084}, {89, 25.29283081429076}, + {90, 25.427452306231707}, {91, 25.34334450564347}, {92, 25.971939771661045}, {93, 25.907910638400836}, {94, 25.971895864171003}, {95, 26.448419417928317}, {96, 26.461057396494514}, {97, 26.332417666691025}, {98, 26.17395667752311}, {99, 25.318121919340044}, + {100, 26.092918888834987}, {101, 26.080903284583307}, {102, 26.65713935644778}, {103, 26.90227335392346}, {104, 27.14904791481757}, {105, 27.45775156727039}, {106, 28.476933177720472}, {107, 27.899526884805073}, {108, 27.274094274806096}, {109, 27.007320452528134}, + {110, 27.767594298696466}, {111, 28.669197835798027}, {112, 28.78287095361503}, {113, 28.744062941208348}, {114, 29.266136201836588}, {115, 29.29631346563548}, {116, 29.30576721224685}, {117, 28.92417256867908}, {118, 29.172020952087326}, {119, 29.36572118529946}, + {120, 29.407882394097168}, {121, 29.24581675687127}, {122, 30.374496989862006}, {123, 29.75867484229172}, {124, 29.724977349500893}, {125, 29.766232845230032}, {126, 29.755425025733203}, {127, 29.891515813029994}, {128, 29.224842590902917}, {129, 29.19894861750696}, + {130, 28.877260649054524}, {131, 29.100160763856657}, {132, 29.11303194254891}, {133, 29.471499370130353}, {134, 29.725430707066725}, {135, 29.875645922022283}, {136, 29.32336837740634}, {137, 29.659223458914788}, {138, 29.916122460150415}, {139, 29.947591268372232}, + {140, 29.912720660976237}, {141, 30.030001305562784}, {142, 30.01603062688162}, {143, 30.206305874273855}, {144, 29.654798308068155}, {145, 29.51005186796388}, {146, 29.479249298103124}, {147, 29.50465455692649}, {148, 28.668390496723326}, {149, 29.70235193454973}, + {150, 29.651621644229916}, {151, 29.514646394709878}, {152, 29.506006233800203}, {153, 29.85094476486748}, {154, 29.815453456016304}, {155, 29.942433471924208}, {156, 29.690047803042003}, {157, 29.081114874079432}, {158, 29.064499954878283}, {159, 29.17427761652929}, + {160, 28.48135454514647}, {161, 28.43614347392514}, {162, 28.615830690136196}, {163, 27.244815449582045}, {164, 28.030839937141593}, {165, 28.09433740820739}, {166, 28.134090510807432}, {167, 28.333335545414524}, {168, 28.187860385951467}, {169, 28.275647207865163}, + {170, 28.110663780904385}, {171, 27.941108395946074}, {172, 27.155839334130572}, {173, 27.486338387798014}, {174, 27.822551655905976}, {175, 27.863745721980674}, {176, 27.51492123912736}, {177, 27.83845450366903}, {178, 27.143346163904276}, {179, 26.470510229480666}, + {180, 27.030260699355054}, {181, 26.530046883178517}, {182, 26.061343714664623}, {183, 26.451944204073293}, {184, 26.430516873002972}, {185, 27.54560649645601}, {186, 27.607455732128237}, {187, 27.3151823435893}, {188, 27.015508353290546}, {189, 27.564104270774138}, + {190, 27.401004407024764}, {191, 27.158236015306873}, {192, 27.15560082391509}, {193, 27.052060260660955}, {194, 27.170854273897024}, {195, 27.764649442110628}, {196, 28.37485762180034}, {197, 28.693644963914128}, {198, 28.639988945921225}, {199, 28.55415913033328}, + {200, 28.227269873938933}, {201, 28.17193587609251}, {202, 28.326544342536973}, {203, 27.64207374329132}, {204, 28.28132114918632}, {205, 28.87895845286964}, {206, 28.188530355189847}, {207, 27.543826104728634}, {208, 27.518668557969406}, {209, 27.501005443892083}, + {210, 27.4034023213343}, {211, 27.81919230719886}, {212, 28.51752545491846}, {213, 28.45642871817993}, {214, 28.281434136230057}, {215, 28.955222450571647}, {216, 28.817758413114476}, {217, 28.512855374048605}, {218, 28.53649013938214}, {219, 28.037741647416063}, + {220, 28.758762367232816}, {221, 28.411225526795217}, {222, 28.081382658095393}, {223, 28.018197966678386}, {224, 27.669840575416934}, {225, 27.729482622798493}, {226, 27.44804950151258}, {227, 27.057357503159633}, {228, 27.02667664651584}, {229, 26.99034079421472}, + {230, 27.624460374012415}, {231, 26.6107079586933}, {232, 26.586632740266342}, {233, 26.50503068257422}, {234, 26.528094947562206}, {235, 27.42586110225183}, {236, 27.58607997646959}, {237, 27.119751888180552}, {238, 27.08687612699868}, {239, 26.50890594997075}, + {240, 27.31126847238326}, {241, 27.241646652930587}, {242, 27.206837899664286}, {243, 27.806910729195003}, {244, 27.943871889687294}, {245, 27.682133734768627}, {246, 27.717567280916626}, {247, 28.50754775235408}, {248, 28.522618646129153}, {249, 28.119450170073687}, + {250, 28.142446798473102}, {251, 27.493875384325104}, {252, 27.232866008532874}, {253, 26.815819408391537}, {254, 26.66806605894335}, {255, 26.832795316319906}, {256, 26.6709072973403}, {257, 26.805339614467922}, {258, 25.87709141573906}, {259, 25.78663848060838}, + {260, 26.292688529709856}, {261, 26.13540568260593}, {262, 26.950679537489023}, {263, 26.74360016328177}, {264, 25.880978465490344}, {265, 26.625349316487267}, {266, 27.726596358113618}, {267, 27.678660195691705}, {268, 27.712915567796777}, {269, 27.721698306913026}, + {270, 27.905267233295028}, {271, 27.957187159455156}, {272, 27.858194094400584}, {273, 28.081932932040928}, {274, 27.735044748607958}, {275, 27.032793497721116}, {276, 27.507939311361685}, {277, 27.574645954142067}, {278, 27.574051095225684}, {279, 27.759761781417595}, + {280, 27.365795276665352}, {281, 26.617667215269847}, {282, 26.798097033417232}, {283, 26.195989054524656}, {284, 26.976938134909243}, {285, 26.89198848346594}, {286, 27.147250351373707}, {287, 27.43948527543756}, {288, 28.0008367927751}, {289, 28.231697069321097}, + {290, 28.7967810312443}, {291, 29.512759843423506}, {292, 29.41555351035952}, {293, 30.70537467842703}, {294, 30.332009655179892}, {295, 30.368538690398843}, {296, 30.73440265654596}, {297, 30.967799052657455}, {298, 31.22392323854322}, {299, 31.122470949743033}, + {300, 31.140432351354303}, {301, 31.13729855237942}, {302, 30.716789453632746}, {303, 31.055606178854195}, {304, 31.025310612429436}, {305, 30.746317663776093}, {306, 31.16989662941391}, {307, 29.623226075091324}, {308, 29.11707721446215}, {309, 29.671462419401518}, + {310, 29.32656872224774}, {311, 29.360034344664133}, {312, 30.193073601940906}, {313, 30.051282853224887}, {314, 30.094809176026143}, {315, 30.62616938793058}, {316, 31.24738345775188}, {317, 31.288951935975373}, {318, 30.793680071961138}, {319, 30.855230722079575}, + {320, 30.172789122099953}, {321, 30.16588226960243}, {322, 30.02896795596743}, {323, 28.710256908027688}, {324, 28.788445957400086}, {325, 29.14084131034792}, {326, 29.154793392113767}, {327, 29.509956518244064}, {328, 29.919766479376083}, {329, 30.809790278228583}, + {330, 31.51994434827819}, {331, 31.14360887134284}, {332, 30.960009728867895}, {333, 31.007757592227232}, {334, 29.912529162927882}, {335, 30.430549132609936}, {336, 30.286270410087386}, {337, 30.29853067098949}, {338, 30.28484502358319}, {339, 30.2649148646112}, + {340, 30.269814084876984}, {341, 29.48743777656781}, {342, 29.25324229055582}, {343, 29.3224085984498}, {344, 28.845749148292715}, {345, 28.25734414947841}, {346, 28.22244460496951}, {347, 28.212727797755832}, {348, 28.099709252761244}, {349, 28.157686542165766}, + {350, 28.15110192590233}, {351, 28.242118337655835}, {352, 29.53367482995351}, {353, 29.532585019641683}, {354, 29.943653842668887}, {355, 28.64740678713178}, {356, 28.627878421859094}, {357, 28.85948017439502}, {358, 29.421837208187565}, {359, 29.213674689162733}, + {360, 29.588524659441383}, {361, 29.871736651370572}, {362, 29.97536581410366}, {363, 29.090848126781193}, {364, 29.07367624649577}, {365, 28.936182688349078}, {366, 28.478980562612755}, {367, 27.701450632319055}, {368, 27.70314463302395}, {369, 27.690633468393905}, + {370, 27.712418470005847}, {371, 28.04360951096915}, {372, 27.899319369636174}, {373, 27.67953057101954}, {374, 27.33713099741098}, {375, 27.715275282261704}, {376, 27.09457980740444}, {377, 26.564904718909652}, {378, 26.851805720484524}, {379, 27.17971899490677}, + {380, 27.203035420946204}, {381, 26.98833921789535}, {382, 27.3376969508508}, {383, 27.36673199620505}, {384, 27.22367712016426}, {385, 27.295774446517406}, {386, 27.24426813335004}, {387, 27.804345559349205}, {388, 27.81107241888124}, {389, 28.007141902081127}, + {390, 28.196204812447014}, {391, 27.570724094739713}, {392, 28.353947742891283}, {393, 28.264285808134655}, {394, 28.001544074943496}, {395, 28.073693599174273}, {396, 28.13948170648531}, {397, 27.7246917685699}, {398, 27.372056917732422}, {399, 26.76664288679182}, + {400, 26.726530330089947}, {401, 26.73659456496834}, {402, 26.35282363879538}, {403, 26.953655414803745}, {404, 27.037728660238816}, {405, 26.391122852260175}, {406, 26.45637305967239}, {407, 25.78486318168484}, {408, 25.92078676697798}, {409, 26.4970571420541}, + {410, 25.86809752383717}, {411, 26.534413662015773}, {412, 27.226357547185447}, {413, 26.71252553894152}, {414, 26.435002451423312}, {415, 26.921455461481504}, {416, 27.06045686220195}, {417, 27.09418382303274}, {418, 28.011098852919908}, {419, 27.896959438752912}, + {420, 27.914042725695705}, {421, 27.547368437091734}, {422, 26.710013475304418}, {423, 26.67332778875601}, {424, 26.624659424882857}, {425, 26.57172379809563}, {426, 26.846697174621756}, {427, 25.8834697613218}, {428, 26.0400273638686}, {429, 25.922770964698366}, + {430, 26.550977646141284}, {431, 26.7196171595444}, {432, 26.57693368973477}, {433, 26.754937488902705}, {434, 26.68923906981248}, {435, 27.578054073124644}, {436, 28.491944955522996}, {437, 29.462067517762534}, {438, 29.049663449716007}, {439, 29.29125398211083}, + {440, 29.716399938469106}, {441, 29.697795912329543}, {442, 28.842426341018918}, {443, 27.995794268014496}, {444, 27.568707173494055}, {445, 27.572194900322412}, {446, 28.94876289646488}, {447, 28.827587778805206}, {448, 28.20122725800809}, {449, 28.982543824618322}, + {450, 28.98764510982713}, {451, 28.476638374330257}, {452, 27.8971845311884}, {453, 28.357599004387694}, {454, 29.550067169716076}, {455, 29.059986913307316}, {456, 29.44698201092504}, {457, 29.444893222605785}, {458, 30.2987283695555}, {459, 30.490753195685443}, + {460, 30.3712818585665}, {461, 31.03092246398632}, {462, 31.1306416649066}, {463, 31.204020755490298}, {464, 31.302761670091414}, {465, 31.287884426895985}, {466, 30.49414678626753}, {467, 30.9842218570926}, {468, 30.88247586217915}, {469, 31.65627890749247}, + {470, 31.300373482363312}, {471, 31.009412915052188}, {472, 31.135319385771062}, {473, 31.142947189076295}, {474, 31.095825728106682}, {475, 31.402733294236572}, {476, 31.348161546083656}, {477, 31.600766880442425}, {478, 31.51732023898528}, {479, 31.218212979786024}, + {480, 31.431310499487758}, {481, 31.536799819023997}, {482, 31.590576295671646}, {483, 31.49429616322668}, {484, 32.1816892011556}, {485, 32.61239327037908}, {486, 32.69902576477615}, {487, 32.89146487332367}, {488, 33.87564290993047}, {489, 33.89045560640269}, + {490, 33.8295470579556}, {491, 33.83070162926511}, {492, 33.57726370546161}, {493, 33.80516778960143}, {494, 34.26212065396207}, {495, 34.4134739195819}, {496, 34.72554255082181}, {497, 34.7082104992009}, {498, 34.60279005425693}, {499, 34.231204914439}, + {500, 34.099389551023904}, {501, 34.22803109888905}, {502, 34.610384365507166}, {503, 34.902573683307146}, {504, 35.0720672663373}, {505, 35.02095234082352}, {506, 35.21029830385259}, {507, 36.39939489896903}, {508, 36.49626843650683}, {509, 36.91759867136517}, + {510, 36.62252163480901}, {511, 36.637052547917165}, {512, 37.07289132044249}, {513, 37.943749079996564}, {514, 37.31678988480502}, {515, 37.10207023840552}, {516, 37.50103188910752}, {517, 36.711556296699854}, {518, 36.69985201593515}, {519, 35.88305508619134}, + {520, 35.819177348929244}, {521, 35.238662967688704}, {522, 35.204473848884604}, {523, 35.73171925218891}, {524, 37.10654099191614}, {525, 36.97832117698606}, {526, 37.88944422270623}, {527, 37.566765945668905}, {528, 36.952108419930035}, {529, 37.16172877345256}, + {530, 36.79432449958657}, {531, 36.83537718548058}, {532, 37.241795032269486}, {533, 36.98172720447497}, {534, 37.12848852369968}, {535, 37.144201171899056}, {536, 37.291084012425635}, {537, 37.20361280452033}, {538, 37.14673041600966}, {539, 36.56539045560211}, + {540, 36.46052858989325}, {541, 36.577035838031236}, {542, 36.413434939980604}, {543, 36.830541819855966}, {544, 36.708601164617306}, {545, 36.754028252130254}, {546, 36.86880281045058}, {547, 36.20522653559122}, {548, 36.19016853939169}, {549, 35.60660078018533}, + {550, 35.58380480655832}, {551, 36.15574897354542}, {552, 35.96603587272573}, {553, 36.01846266557714}, {554, 36.1246059100681}, {555, 35.776226062342324}, {556, 35.53756438274776}, {557, 36.09285691162272}, {558, 36.024821241563764}, {559, 36.07242575850486}, + {560, 36.902047436744915}, {561, 36.783371618493575}, {562, 36.97312505169229}, {563, 38.33987731208528}, {564, 38.12211125979531}, {565, 37.867344065570286}, {566, 37.91294220403035}, {567, 37.90421561454514}, {568, 37.52932323225818}, {569, 38.09065950561944}, + {570, 38.3481668904528}, {571, 38.38853417076884}, {572, 38.82875488776441}, {573, 39.30100206552399}, {574, 39.22872668781665}, {575, 39.26377992972712}, {576, 39.11286987148877}, {577, 38.55996270262097}, {578, 39.53759984840164}, {579, 39.434972831723044}, + {580, 39.58456959406775}, {581, 40.02469156333131}, {582, 39.83799518416734}, {583, 41.066310308184526}, {584, 41.249674158569356}, {585, 41.84405324758257}, {586, 42.44197850928179}, {587, 42.44435586698777}, {588, 42.51450858652831}, {589, 42.497079884180856}, + {590, 41.86201338457368}, {591, 41.97430910917171}, {592, 42.38604395890263}, {593, 42.77701512105045}, {594, 42.917712727399795}, {595, 41.92733574585986}, {596, 41.98001270512809}, {597, 42.21709441156598}, {598, 41.73101118068132}, {599, 41.511566903501254}, + {600, 41.151843749654674}, {601, 40.936735208747976}, {602, 41.07670022261416}, {603, 41.00310883262705}, {604, 41.175657930592564}, {605, 39.65057441921113}, {606, 40.59197307871199}, {607, 40.19328008308628}, {608, 39.96642122821799}, {609, 39.980554983282815}, + {610, 39.198802516126015}, {611, 40.28881956306739}, {612, 40.835001630650424}, {613, 40.96840019049191}, {614, 41.47803284199829}, {615, 42.31117692587696}, {616, 42.56501744522036}, {617, 40.75017426985965}, {618, 40.54474737773533}, {619, 40.32299207284596}, + {620, 40.59233541049153}, {621, 39.799369403233875}, {622, 39.90740256156506}, {623, 39.84656589770128}, {624, 40.51863738080248}, {625, 40.36927897222197}, {626, 39.577466020927666}, {627, 40.24059624379228}, {628, 39.888654784809695}, {629, 39.06459566315487}, + {630, 39.21824625899819}, {631, 39.707949621228416}, {632, 39.6862655989382}, {633, 39.232670266668386}, {634, 38.72226788949448}, {635, 38.90143970392931}, {636, 38.58114412526509}, {637, 37.47764070721714}, {638, 38.14287868141234}, {639, 37.95489640397961}, + {640, 37.978479310316544}, {641, 38.08870157682401}, {642, 37.86650279398312}, {643, 37.83158400391423}, {644, 37.661339861002425}, {645, 37.77028468310174}, {646, 38.19873503550377}, {647, 37.84181306063926}, {648, 36.15953352691078}, {649, 36.15954745676219}, + {650, 36.70039654101666}, {651, 36.747404928048674}, {652, 36.45440618086992}, {653, 36.50375344553345}, {654, 36.12499971175383}, {655, 36.25540639161984}, {656, 36.14128341022631}, {657, 36.13550126300937}, {658, 36.47636290489859}, {659, 36.52934581511136}, + {660, 36.180684191314654}, {661, 36.1345774586021}, {662, 35.88734753480093}, {663, 35.910114094303935}, {664, 37.494624894014834}, {665, 37.388322481490356}, {666, 37.51949607161456}, {667, 37.42672416426224}, {668, 37.607431895150505}, {669, 38.1169043451165}, + {670, 38.198701944392226}, {671, 37.672228180889036}, {672, 37.573597996143555}, {673, 37.61475549631909}, {674, 37.704351969731974}, {675, 36.526503465131775}, {676, 37.14283711760895}, {677, 37.042216595367044}, {678, 37.19943347657908}, {679, 37.030753837557455}, + {680, 37.29832109446188}, {681, 37.34909191845969}, {682, 37.396667764315495}, {683, 37.71513970238973}, {684, 37.035963822400284}, {685, 37.09132966834279}, {686, 36.671363246669195}, {687, 36.725271547538746}, {688, 36.732765304609515}, {689, 37.049756926857675}, + {690, 37.100425916489115}, {691, 37.80849557841213}, {692, 38.00370168917147}, {693, 38.65064301128746}, {694, 38.16137008804746}, {695, 38.16628690738749}, {696, 38.509868983636345}, {697, 38.51329094834429}, {698, 38.63409846835321}, {699, 38.33551139578646}, + {700, 37.86178126970245}, {701, 37.20625176875522}, {702, 37.13864013016608}, {703, 36.598956314019325}, {704, 36.73225170870823}, {705, 36.645909925459954}, {706, 36.931378033672665}, {707, 37.15159698187774}, {708, 37.20499450218726}, {709, 37.312534314076736}, + {710, 38.234122213566266}, {711, 38.23482019038691}, {712, 39.50821685803287}, {713, 39.3687472311753}, {714, 40.43514365409954}, {715, 40.44682389295688}, {716, 39.81067959969004}, {717, 39.4239320658717}, {718, 39.54620136509768}, {719, 39.70807547297041}, + {720, 39.66237013592385}, {721, 39.72023820319751}, {722, 40.02829005394066}, {723, 39.9923106822397}, {724, 39.88229973919716}, {725, 39.577778632304906}, {726, 39.42713453659046}, {727, 39.43664948339558}, {728, 39.90314312163109}, {729, 40.00741574107013}, + {730, 40.7177664595581}, {731, 40.72045099279074}, {732, 38.3054003174184}, {733, 37.63394119070095}, {734, 37.190128597342806}, {735, 37.63802392396217}, {736, 37.09659077679385}, {737, 37.08430082665866}, {738, 37.031926029911546}, {739, 36.92762266554352}, + {740, 36.768283960686276}, {741, 36.80986118421022}, {742, 37.00023584840072}, {743, 37.37757227329164}, {744, 37.65237108287256}, {745, 37.65431119266612}, {746, 37.78202911987313}, {747, 37.26755576516154}, {748, 37.253878362362926}, {749, 37.25181149353707}, + {750, 38.046853987139436}, {751, 38.52088550994239}, {752, 38.50060736503108}, {753, 39.721168829902766}, {754, 39.714977806533284}, {755, 41.2027128451164}, {756, 41.271353835056736}, {757, 41.42756394972893}, {758, 41.20791889996719}, {759, 41.51542890247571}, + {760, 41.533271125006856}, {761, 41.57431095381387}, {762, 42.43297879649776}, {763, 42.697486616671064}, {764, 42.943817792210965}, {765, 43.00326844639773}, {766, 43.04122373477804}, {767, 42.72385932125831}, {768, 42.91563938083615}, {769, 43.09789890970447}, + {770, 43.21939147903337}, {771, 42.593500985806116}, {772, 42.552074422506266}, {773, 41.96625047640962}, {774, 42.43760021996765}, {775, 42.15036608724706}, {776, 42.01801170238841}, {777, 42.108251206065354}, {778, 42.13720797898185}, {779, 41.734510693754125}, + {780, 41.98861197273539}, {781, 41.855890804590906}, {782, 41.83653956517087}, {783, 41.84729249879647}, {784, 41.34451591903922}, {785, 41.198937855641844}, {786, 40.9934126996872}, {787, 40.33193886547389}, {788, 40.42177093298795}, {789, 40.56423832118387}, + {790, 40.56930078212437}, {791, 40.44001792153413}, {792, 39.36239492015202}, {793, 39.2299512438198}, {794, 39.98930528782557}, {795, 39.78753261124546}, {796, 39.7771218730521}, {797, 40.08078557739916}, {798, 39.9026881045777}, {799, 39.846598822934695}, + {800, 39.819165988548605}, {801, 39.779536346223246}, {802, 39.417184051357914}, {803, 39.04248228046798}, {804, 39.04467471746209}, {805, 39.19976668706493}, {806, 38.62608414650186}, {807, 38.56950606795273}, {808, 38.11427821927686}, {809, 38.003564811967195}, + {810, 38.9186775339263}, {811, 39.32280711240079}, {812, 39.37066040646152}, {813, 39.35771550162152}, {814, 39.82593927687213}, {815, 39.41511912781052}, {816, 39.632089125382805}, {817, 39.7013716770543}, {818, 39.71871324205181}, {819, 39.3691037472924}, + {820, 39.4630528375144}, {821, 39.463557590573124}, {822, 39.84149848240294}, {823, 39.731112944493475}, {824, 39.46553273230596}, {825, 38.983001317774516}, {826, 39.111806633972954}, {827, 39.11241986582127}, {828, 39.060815486719235}, {829, 38.62106612409818}, + {830, 38.45726527644363}, {831, 38.46243114268653}, {832, 37.81089603641852}, {833, 37.06489698994548}, {834, 35.65801215219413}, {835, 35.598044467413054}, {836, 35.51269552874315}, {837, 35.26479970268285}, {838, 36.55531335895278}, {839, 36.707855369730034}, + {840, 36.5489032234404}, {841, 36.3485272462068}, {842, 36.365162732119266}, {843, 36.175883792606705}, {844, 36.16643352096541}, {845, 36.721302046157795}, {846, 36.738751647960406}, {847, 36.499637035175965}, {848, 36.461882187600125}, {849, 36.474313661197286}, + {850, 36.138203284641186}, {851, 36.47458889999187}, {852, 36.9122685068765}, {853, 36.84193719836966}, {854, 36.016869149964144}, {855, 35.908626743361204}, {856, 35.853057192524155}, {857, 35.49400703601318}, {858, 35.23328368215224}, {859, 35.43381027728788}, + {860, 35.441912396566295}, {861, 36.11980562386126}, {862, 35.987081872648865}, {863, 35.354742046128464}, {864, 35.11437170180045}, {865, 35.10075744627144}, {866, 35.33061979174065}, {867, 35.26826901332004}, {868, 34.76420766370852}, {869, 34.36825504722903}, + {870, 33.95310090031108}, {871, 34.76854906618263}, {872, 34.96941220214851}, {873, 34.89082477498721}, {874, 34.930226673548425}, {875, 34.818344629526955}, {876, 33.46752591731415}, {877, 32.4461932776035}, {878, 32.63093881621515}, {879, 32.571549274273316}, + {880, 31.70273743751098}, {881, 31.560570134656707}, {882, 30.11896242237412}, {883, 29.976753491813355}, {884, 30.151470049420453}, {885, 29.968667652811302}, {886, 29.914108861900175}, {887, 29.315297425726456}, {888, 29.465096537128055}, {889, 30.38412313034993}, + {890, 30.529005936656326}, {891, 30.377111661676715}, {892, 30.690112657456975}, {893, 31.19665096563887}, {894, 30.23180166165202}, {895, 30.214575260830543}, {896, 30.61887791868095}, {897, 31.69516453152183}, {898, 31.724657127738052}, {899, 31.3260343161325}, + {900, 31.448576476567858}, {901, 31.384197051367188}, {902, 31.42382604514501}, {903, 31.77933470325809}, {904, 31.54527210384609}, {905, 31.662859065404803}, {906, 31.952233276070345}, {907, 31.710429406503827}, {908, 31.69910928793987}, {909, 31.26054806750292}, + {910, 31.252913740901814}, {911, 31.069356216279907}, {912, 31.450376052945543}, {913, 31.446602308578885}, {914, 32.88848724097737}, {915, 32.97405474992589}, {916, 32.95989543002121}, {917, 33.50228160406862}, {918, 33.57376811625785}, {919, 33.49455873288756}, + {920, 32.739675795181945}, {921, 32.69683391724602}, {922, 33.81401934792278}, {923, 33.68504119872676}, {924, 33.08473835369315}, {925, 32.71783936639953}, {926, 33.576629314879405}, {927, 33.496669377208725}, {928, 32.32323980202774}, {929, 32.19540083945545}, + {930, 31.681562294745188}, {931, 31.21936553988134}, {932, 32.02350301138198}, {933, 31.99391696064781}, {934, 31.222294641434715}, {935, 31.49393327555392}, {936, 31.60562877897029}, {937, 31.719747501964054}, {938, 31.631337228109494}, {939, 31.876599195053196}, + {940, 31.839975677295737}, {941, 31.57998061624259}, {942, 32.247156373569744}, {943, 33.13453225404717}, {944, 33.57148226581986}, {945, 33.67881606804257}, {946, 34.27627744671606}, {947, 34.2667236377486}, {948, 35.0394972266412}, {949, 35.06221556029825}, + {950, 35.317831610972426}, {951, 35.47650956265658}, {952, 35.92884652798027}, {953, 36.40171114341679}, {954, 35.651733034914436}, {955, 36.021038886103796}, {956, 36.194620959224366}, {957, 36.14405991221338}, {958, 36.02271118328351}, {959, 36.01617792703485}, + {960, 36.718644368420634}, {961, 36.658954226071046}, {962, 36.58675202460727}, {963, 35.89078679017791}, {964, 35.94749384377098}, {965, 35.92838317262612}, {966, 34.32490064912566}, {967, 33.68329350874763}, {968, 33.72780060374807}, {969, 33.521931400031036}, + {970, 33.48546378484414}, {971, 33.729671379093105}, {972, 32.483138764110244}, {973, 32.70061143845538}, {974, 31.882058598338592}, {975, 31.85978103716163}, {976, 31.360355110091252}, {977, 31.338506662447838}, {978, 31.4975196000238}, {979, 31.636516566035457}, + {980, 31.76928554942759}, {981, 31.596565826225252}, {982, 31.24504603522091}, {983, 30.97685607999551}, {984, 30.776149614114455}, {985, 30.369022511891757}, {986, 30.65823464304316}, {987, 30.536961423755283}, {988, 30.543171096017304}, {989, 29.709033348016842}, + {990, 29.787775153315135}, {991, 30.020098011459627}, {992, 30.161570003700696}, {993, 30.110594653721783}, {994, 30.269521765071406}, {995, 30.887702552351005}, {996, 31.092501292990715}, {997, 30.96235060201595}, {998, 31.371052725132344}, {999, 31.652461552974707}, + {1000, 31.925382030352395}, {1001, 32.09718279718192}, {1002, 32.404705995647134}, {1003, 32.40399854679425}, {1004, 32.105619052543034}, {1005, 32.123902211392696}, {1006, 32.26894930916644}, {1007, 32.06571185804976}, {1008, 32.27138533885846}, {1009, 32.459468377391936}, + {1010, 32.45637104295923}, {1011, 33.97355416481095}, {1012, 34.073372443262755}, {1013, 34.555509854599876}, {1014, 34.001992433795586}, {1015, 34.028775375039515}, {1016, 33.15444517016032}, {1017, 33.25999551152861}, {1018, 33.56889630842039}, {1019, 33.836008073592865}, + {1020, 34.6557254577998}, {1021, 34.16915786529838}, {1022, 33.91355886515464}, {1023, 33.53814944935212}, {1024, 33.73093246034526}, {1025, 33.779257221704746}, {1026, 33.55901656076105}, {1027, 33.522411543634334}, {1028, 33.308377022855574}, {1029, 33.46872487000558}, + {1030, 33.398428332372106}, {1031, 32.78130678002823}, {1032, 32.569113758452666}, {1033, 31.58571790720343}, {1034, 32.16172427938117}, {1035, 31.661979621179476}, {1036, 31.632175065844496}, {1037, 31.789546741031184}, {1038, 32.26513372369237}, {1039, 32.18304328830448}, + {1040, 32.71362639662142}, {1041, 31.299268596529224}, {1042, 27.792537058477052}, {1043, 31.385259077931753}, {1044, 30.591065858607116}, {1045, 31.272338876555935}, {1046, 31.673417165689862}, {1047, 31.185485416617453}, {1048, 31.29116101415942}, {1049, 31.420045699024275}, + {1050, 32.72027856671056}, {1051, 32.70728464282712}, {1052, 32.84488754596194}, {1053, 33.19971370723099}, {1054, 33.00087402494973}, {1055, 32.99077340899255}, {1056, 32.83232129107784}, {1057, 32.30498262311541}, {1058, 32.30946094376116}, {1059, 32.940891339151776}, + {1060, 32.68854192681414}, {1061, 32.73363409330329}, {1062, 32.79589758581314}, {1063, 32.289791532294295}, {1064, 32.68604720099993}, {1065, 33.33078243465505}, {1066, 33.54826648297002}, {1067, 33.40865566016796}, {1068, 33.41296471315078}, {1069, 32.77182145012062}, + {1070, 32.33224727266914}, {1071, 32.27203157051601}, {1072, 32.26109905266389}, {1073, 32.921499096795074}, {1074, 33.38395175893338}, {1075, 33.78768855190261}, {1076, 33.442367223026764}, {1077, 33.237913854994794}, {1078, 31.147232293821737}, {1079, 31.146482126046756}, + {1080, 31.71150780117646}, {1081, 31.814296312484636}, {1082, 30.955859429080363}, {1083, 30.987259649137073}, {1084, 30.89065910781576}, {1085, 31.16579586461409}, {1086, 31.118733218136573}, {1087, 30.533273297600925}, {1088, 29.252834071159857}, {1089, 29.685743434805644}, + {1090, 29.705920523322913}, {1091, 29.85069842502032}, {1092, 30.388305038615574}, {1093, 30.161758590806624}, {1094, 30.779652282941164}, {1095, 30.91190072880913}, {1096, 31.56528710281942}, {1097, 31.768065818580393}, {1098, 31.8230334699979}, {1099, 32.1658225590106}, + {1100, 32.18402247418236}, {1101, 33.487267406897004}, {1102, 33.51902089155666}, {1103, 33.39833431034105}, {1104, 32.50713483623776}, {1105, 32.66530621931014}, {1106, 32.78927126650209}, {1107, 32.650964726546256}, {1108, 32.70706148431291}, {1109, 32.916943844199004}, + {1110, 32.49004088753038}, {1111, 32.945619481604744}, {1112, 33.03101393088018}, {1113, 33.01159487630862}, {1114, 33.17414749212253}, {1115, 34.21219988578758}, {1116, 34.00043238845383}, {1117, 33.61923485510282}, {1118, 34.275948039809364}, {1119, 33.87944072377637}, + {1120, 33.57361872056542}, {1121, 33.60777850243776}, {1122, 33.95519829789812}, {1123, 34.101866130091636}, {1124, 35.08344176854378}, {1125, 35.290045657998256}, {1126, 34.99789523160777}, {1127, 35.11162512342385}, {1128, 35.150203027075506}, {1129, 34.9547553234651}, + {1130, 35.68267560816188}, {1131, 35.94530333733083}, {1132, 36.40684420002165}, {1133, 36.90897436886569}, {1134, 36.68925056413046}, {1135, 36.77241779692453}, {1136, 36.7768373962142}, {1137, 36.615559524302526}, {1138, 36.15108571210806}, {1139, 36.31887166999472}, + {1140, 36.25359696272827}, {1141, 36.04048635034041}, {1142, 36.00631287066912}, {1143, 36.459483786091326}, {1144, 36.523711914237204}, {1145, 35.76502964124057}, {1146, 36.18603634547918}, {1147, 36.143227345894964}, {1148, 35.42846972819206}, {1149, 34.22551552384586}, + {1150, 35.085356710729094}, {1151, 34.978408345953554}, {1152, 35.02162912627897}, {1153, 34.6749544956021}, {1154, 34.5294083257531}, {1155, 33.989361791250865}, {1156, 33.90560409554212}, {1157, 34.22819180059112}, {1158, 35.09979866464593}, {1159, 35.459597394271825}, + {1160, 35.028413870661275}, {1161, 35.07365163864229}, {1162, 35.008455674695846}, {1163, 34.87027987153975}, {1164, 32.73740982908738}, {1165, 35.86288920810587}, {1166, 36.38448730256452}, {1167, 35.69198792205293}, {1168, 35.7287802068078}, {1169, 35.67047911791235}, + {1170, 35.65931562676154}, {1171, 35.63860255804954}, {1172, 36.28377914027185}, {1173, 36.31724433582806}, {1174, 36.42873315721097}, {1175, 36.55293797881833}, {1176, 36.68858364718202}, {1177, 36.68542383419321}, {1178, 36.59630640724288}, {1179, 36.46919978323624}, + {1180, 35.97612499492244}, {1181, 35.97470028204401}, {1182, 36.88186754506275}, {1183, 36.89180762556425}, {1184, 37.398624973193115}, {1185, 37.6485574658129}, {1186, 37.59944793075436}, {1187, 37.52123044476694}, {1188, 37.545545238118}, {1189, 37.63568398432097}, + {1190, 37.60117292673316}, {1191, 37.76737303107856}, {1192, 37.796708933792935}, {1193, 38.44788884052119}, {1194, 38.433526334130406}, {1195, 38.53261361058563}, {1196, 38.20625193191746}, {1197, 38.216892858955696}, {1198, 38.10624817074387}, {1199, 38.31498368822061}, + {1200, 38.28171670233931}, {1201, 38.03504598890333}, {1202, 37.469381149070585}, {1203, 37.645843865994834}, {1204, 37.996863546861626}, {1205, 37.39010863231819}, {1206, 37.916484620097826}, {1207, 37.9792008265341}, {1208, 37.56402382212345}, {1209, 37.64236828858301}, + {1210, 38.20048604682538}, {1211, 38.1135776386156}, {1212, 38.06926736333271}, {1213, 37.85109980183144}, {1214, 37.8667193253515}, {1215, 37.79731989239691}, {1216, 37.79569290509785}, {1217, 37.507680665850124}, {1218, 37.86538998455543}, {1219, 37.97413406343451}, + {1220, 38.153189871670634}, {1221, 37.705945919911386}, {1222, 38.003018638410715}, {1223, 37.97804975659373}, {1224, 37.86005819640923}, {1225, 37.74622347438676}, {1226, 37.24273013769576}, {1227, 37.471847978186396}, {1228, 36.574577925568015}, {1229, 36.898112403986175}, + {1230, 37.93262825928585}, {1231, 37.98350758652906}, {1232, 37.3327630080204}, {1233, 37.87253628352105}, {1234, 37.03465628886385}, {1235, 37.3628779234913}, {1236, 37.257920892232235}, {1237, 37.355020597223806}, {1238, 36.42612381389829}, {1239, 35.71849960760021}, + {1240, 35.569737001180215}, {1241, 35.639328778372764}, {1242, 36.67505414760637}, {1243, 36.529877025906195}, {1244, 36.17805747298349}, {1245, 36.20730061152778}, {1246, 36.7654190000416}, {1247, 36.066963570873966}, {1248, 34.80315462413555}, {1249, 34.79204497446974}, + {1250, 34.96282416930701}, {1251, 35.29525328571966}, {1252, 35.39276674967225}, {1253, 35.17789113339765}, {1254, 35.1734239698592}, {1255, 36.27186268761327}, {1256, 36.12969032844646}, {1257, 35.56797468172719}, {1258, 35.413952983479305}, {1259, 35.473554398381665}, + {1260, 34.959780400150386}, {1261, 34.705612847987084}, {1262, 35.122081505219015}, {1263, 35.54647234411569}, {1264, 35.56585165590874}, {1265, 34.07163330273662}, {1266, 34.061764136955}, {1267, 34.98718482740182}, {1268, 35.194478377248124}, {1269, 34.814779163097306}, + {1270, 35.63339066284622}, {1271, 36.248018614293784}, {1272, 35.89931722172987}, {1273, 36.02707673294714}, {1274, 35.47928825105021}, {1275, 35.339226659095814}, {1276, 36.04365822060287}, {1277, 36.12975065194946}, {1278, 35.657328019340454}, {1279, 35.92456974548614}, + {1280, 36.330656914157935}, {1281, 36.192088120039486}, {1282, 35.48734879533039}, {1283, 35.20967589696835}, {1284, 35.16528283177774}, {1285, 35.05450151673027}, {1286, 34.083660774124404}, {1287, 33.772468853879964}, {1288, 33.29191351706884}, {1289, 33.40417208662001}, + {1290, 32.607679572698984}, {1291, 32.60800666862871}, {1292, 32.6060770265256}, {1293, 32.50260313437235}, {1294, 32.66708354095242}, {1295, 32.66682027747426}, {1296, 33.0078376785496}, {1297, 32.79448909601}, {1298, 33.33055451566498}, {1299, 33.80720872380922}, + {1300, 34.03662018610426}, {1301, 33.471561770605334}, {1302, 33.464542278200675}, {1303, 33.5135192502925}, {1304, 33.522105679079196}, {1305, 33.486732091763336}, {1306, 34.102696294452976}, {1307, 34.28410402104561}, {1308, 34.50821094970082}, {1309, 34.2800121277366}, + {1310, 34.07699583444763}, {1311, 33.29032898991077}, {1312, 33.447416548419994}, {1313, 33.07070289905697}, {1314, 32.76441433179846}, {1315, 32.35364095869243}, {1316, 32.19453136744108}, {1317, 32.001227569765504}, {1318, 32.1893553685659}, {1319, 32.17192961357054}, + {1320, 32.05156175088731}, {1321, 32.01319521995405}, {1322, 31.361540797597904}, {1323, 31.358737048716332}, {1324, 31.4094163556713}, {1325, 31.385674150907523}, {1326, 30.397821572277874}, {1327, 30.430993318308072}, {1328, 30.559151734617863}, {1329, 30.709052945784077}, + {1330, 30.762480629193092}, {1331, 30.428823245671445}, {1332, 30.549369360601233}, {1333, 30.401524153538364}, {1334, 30.862514784963658}, {1335, 30.543179751483258}, {1336, 30.546876105280212}, {1337, 30.8118556608746}, {1338, 30.73308277586123}, {1339, 31.09586849152352}, + {1340, 30.443543543528172}, {1341, 30.574438626388865}, {1342, 30.433112514559557}, {1343, 29.60468065056554}, {1344, 29.518027759834045}, {1345, 29.530716732475156}, {1346, 29.258574444161773}, {1347, 28.96213974116116}, {1348, 30.094589411608464}, {1349, 30.027880530088794}, + {1350, 30.00173657659883}, {1351, 30.7570211333231}, {1352, 30.905455874138642}, {1353, 30.95776014140416}, {1354, 31.25924961163502}, {1355, 31.28781614068638}, {1356, 31.259224092967646}, {1357, 31.471490532081248}, {1358, 30.66993696457886}, {1359, 29.51601931381402}, + {1360, 29.658359527955362}, {1361, 29.498776144228817}, {1362, 29.0952884352307}, {1363, 27.583844233928247}, {1364, 27.104268459905846}, {1365, 25.95144210780829}, {1366, 25.823464503416666}, {1367, 25.72514162857761}, {1368, 25.654123815674225}, {1369, 26.051979147188657}, + {1370, 25.841597348616787}, {1371, 25.03088000259754}, {1372, 24.40430526113091}, {1373, 24.39450861715055}, {1374, 24.868258110961357}, {1375, 24.87891945990089}, {1376, 24.826829203959203}, {1377, 24.723127466527284}, {1378, 25.189101108151846}, {1379, 24.999892026613335}, + {1380, 25.403952465657824}, {1381, 25.395818912419763}, {1382, 25.361488701260853}, {1383, 25.168837708367573}, {1384, 25.822974064286573}, {1385, 25.184633998151693}, {1386, 25.673392593447062}, {1387, 25.493115319468433}, {1388, 25.616885562952366}, {1389, 26.26185502334556}, + {1390, 26.63671721716633}, {1391, 27.41659614028529}, {1392, 27.180001027499763}, {1393, 27.31191586358311}, {1394, 27.41516507193562}, {1395, 27.621542573182392}, {1396, 28.333329103903207}, {1397, 28.490364050694232}, {1398, 28.282737229661876}, {1399, 28.487368411327616}, + {1400, 27.619991904229085}, {1401, 27.775228746290942}, {1402, 27.75784788858771}, {1403, 26.388008503362745}, {1404, 26.390446759456594}, {1405, 26.804841863728445}, {1406, 26.645161583486924}, {1407, 25.978560150820044}, {1408, 25.928009125329705}, {1409, 25.928237301882948}, + {1410, 25.750677096758384}, {1411, 25.3386270250864}, {1412, 25.140774560316675}, {1413, 24.34074260607449}, {1414, 24.754961425981048}, {1415, 24.730194413283783}, {1416, 24.920450795877183}, {1417, 25.024801997124456}, {1418, 24.6651943693905}, {1419, 25.194960339720506}, + {1420, 25.34229435510204}, {1421, 25.305380825552025}, {1422, 25.412841036914795}, {1423, 25.505668659821385}, {1424, 25.930434241278466}, {1425, 25.958267633145237}, {1426, 26.031451433938873}, {1427, 26.586990196306772}, {1428, 27.292096678546}, {1429, 28.12071488383503}, + {1430, 28.02832144500101}, {1431, 28.58386148422691}, {1432, 28.79404266374261}, {1433, 29.738295917981866}, {1434, 29.720007389392677}, {1435, 29.63633778940114}, {1436, 29.68841036823205}, {1437, 29.575033122960512}, {1438, 29.539149897548384}, {1439, 29.80686249903287}, + {1440, 30.862182502538953}, {1441, 30.42493141449586}, {1442, 30.427703473898646}, {1443, 30.502760352966796}, {1444, 30.537509584235174}, {1445, 30.206448675310977}, {1446, 30.230172108384416}, {1447, 30.164364772985248}, {1448, 29.417259389108583}, {1449, 29.280495127154293}, + {1450, 29.331776246449813}, {1451, 28.98662862990845}, {1452, 29.752510306571484}, {1453, 29.70066010243845}, {1454, 29.357272975382973}, {1455, 29.25016590767666}, {1456, 29.355245117563307}, {1457, 29.533981898101043}, {1458, 29.569017902727573}, {1459, 29.611976373457114}, + {1460, 29.731805290138638}, {1461, 29.757994213197602}, {1462, 30.0901622202274}, {1463, 31.279393484956277}, {1464, 31.228915684424805}, {1465, 31.39987446802975}, {1466, 30.81523244839042}, {1467, 31.19319811379786}, {1468, 31.21869066135987}, {1469, 31.378064726660238}, + {1470, 31.36503448737635}, {1471, 30.890723656435668}, {1472, 31.278271018448063}, {1473, 31.27351266944355}, {1474, 30.906433524045895}, {1475, 31.094199381637562}, {1476, 32.22446329349785}, {1477, 31.755676495678124}, {1478, 31.863329580086756}, {1479, 34.1769485077606}, + {1480, 33.67109006952487}, {1481, 34.032560599893785}, {1482, 35.17463593455005}, {1483, 34.68680491085987}, {1484, 35.34323186904023}, {1485, 36.06846197316916}, {1486, 35.75474567916718}, {1487, 34.95879681397107}, {1488, 34.91513272089497}, {1489, 34.27429545432343}, + {1490, 34.47677587538645}, {1491, 34.8837191993053}, {1492, 34.888682229883216}, {1493, 34.6615444728743}, {1494, 34.691697874937915}, {1495, 34.673137205747096}, {1496, 34.619588001105676}, {1497, 34.67924802646811}, {1498, 34.711467226688804}, {1499, 34.378496298766606}, + {1500, 34.01906505065623}, {1501, 34.1921149855785}, {1502, 34.645939695035565}, {1503, 34.55325778595661}, {1504, 34.837013970080854}, {1505, 34.3201819146262}, {1506, 34.21433963893517}, {1507, 34.209807913410046}, {1508, 34.29503290261106}, {1509, 34.372533557964495}, + {1510, 34.29249674028179}, {1511, 34.1793952178106}, {1512, 34.23027204162766}, {1513, 34.30024655177506}, {1514, 34.29076290991813}, {1515, 34.872527881393644}, {1516, 34.88183138613434}, {1517, 34.99793065640857}, {1518, 35.116718830483286}, {1519, 35.11046533465197}, + {1520, 36.014438556584366}, {1521, 36.02360905142489}, {1522, 35.91482043384545}, {1523, 35.809274512011065}, {1524, 35.59619773000511}, {1525, 35.44019543122139}, {1526, 35.64144780604629}, {1527, 35.58080982359661}, {1528, 35.452624932745614}, {1529, 35.467238834711935}, + {1530, 35.041073967062886}, {1531, 34.27022167037798}, {1532, 33.77867219668047}, {1533, 33.675446148880425}, {1534, 34.29649579312852}, {1535, 33.80063094230085}, {1536, 33.83706673096555}, {1537, 34.19813699318839}, {1538, 34.67898341154741}, {1539, 34.61565170828406}, + {1540, 34.88435222539337}, {1541, 34.74803849982474}, {1542, 34.88836986733776}, {1543, 34.316664346866176}, {1544, 35.1429379016867}, {1545, 35.236518386423754}, {1546, 35.480022507695814}, {1547, 35.51129848892317}, {1548, 35.55074228557621}, {1549, 35.47965462275921}, + {1550, 35.1614123321548}, {1551, 34.649935916461956}, {1552, 33.831408529297455}, {1553, 33.68659149830347}, {1554, 34.76348070802675}, {1555, 34.75436175642519}, {1556, 34.670519236256325}, {1557, 34.40743801250581}, {1558, 35.03821600965653}, {1559, 35.074059881857586}, + {1560, 34.66313179952048}, {1561, 34.05122520128333}, {1562, 34.369261269504776}, {1563, 34.306862410511414}, {1564, 34.35513437341248}, {1565, 34.42016699549108}, {1566, 34.32411882750936}, {1567, 35.10444315366103}, {1568, 35.21667415434252}, {1569, 34.56235394065422}, + {1570, 34.99569538327316}, {1571, 35.051462529787685}, {1572, 35.65901988130965}, {1573, 35.30919470315533}, {1574, 35.43427648307305}, {1575, 35.26105873637407}, {1576, 34.984210066219426}, {1577, 34.974915529232504}, {1578, 34.64250053081252}, {1579, 34.855051142104344}, + {1580, 34.61935447447808}, {1581, 33.30095950674964}, {1582, 33.52159175029287}, {1583, 33.30245847520868}, {1584, 33.22017010546928}, {1585, 33.302509858225584}, {1586, 33.37904181575153}, {1587, 33.25298440074963}, {1588, 33.25372890621039}, {1589, 33.55633290212615}, + {1590, 33.564240306946566}, {1591, 33.77554531353173}, {1592, 33.77747427631988}, {1593, 33.758375775192526}, {1594, 34.36417258641864}, {1595, 34.31644882079321}, {1596, 33.55182639869085}, {1597, 33.56219047110169}, {1598, 33.44110230199712}, {1599, 33.37853776463871}, + {1600, 32.03520633056414}, {1601, 32.091659309762015}, {1602, 32.3728274207352}, {1603, 32.32296532348558}, {1604, 32.248876392188286}, {1605, 32.2486900311155}, {1606, 32.30827587355294}, {1607, 31.55600150552031}, {1608, 32.275031428073206}, {1609, 32.27879842764017}, + {1610, 32.12685506952653}, {1611, 32.04232751622579}, {1612, 32.3717511617595}, {1613, 32.90499426727991}, {1614, 31.34160341506861}, {1615, 32.79935984939758}, {1616, 33.3133749284244}, {1617, 33.330066949919654}, {1618, 33.6696207424472}, {1619, 33.44727416894953}, + {1620, 32.937035354917406}, {1621, 32.42098193976603}, {1622, 32.40919020921572}, {1623, 32.40926615430823}, {1624, 31.91939496950345}, {1625, 31.745665996112034}, {1626, 31.762245569926346}, {1627, 31.76510278674478}, {1628, 32.18895805255132}, {1629, 32.17884231058142}, + {1630, 31.634810462741328}, {1631, 31.569967535436145}, {1632, 32.29578832893024}, {1633, 32.930732352651475}, {1634, 33.03934988366926}, {1635, 33.21635409643862}, {1636, 33.261326384884754}, {1637, 33.39655980314689}, {1638, 35.11945078307172}, {1639, 35.076572468626274}, + {1640, 34.95856618844892}, {1641, 34.85019550097897}, {1642, 34.70787112679453}, {1643, 34.64396731467868}, {1644, 34.03523254807719}, {1645, 34.599545249438016}, {1646, 34.911136115744284}, {1647, 35.12923123083873}, {1648, 34.19771898956377}, {1649, 34.081163717338875}, + {1650, 33.952021163589286}, {1651, 34.03909563960654}, {1652, 33.826004075423334}, {1653, 33.69501183562262}, {1654, 33.54101912632574}, {1655, 33.534398838396605}, {1656, 33.551479085175345}, {1657, 34.17014679283176}, {1658, 33.68298225679782}, {1659, 33.914956135823154}, + {1660, 33.85350696847641}, {1661, 33.5930418617416}, {1662, 33.698585740003914}, {1663, 33.53164709646962}, {1664, 32.942749249597995}, {1665, 32.78927293361512}, {1666, 32.805622975726415}, {1667, 32.678834731539396}, {1668, 32.49005865138826}, {1669, 32.45598674269622}, + {1670, 32.38305018611963}, {1671, 32.39849660167694}, {1672, 33.26082532734355}, {1673, 33.32014170105489}, {1674, 33.73004210039798}, {1675, 33.724298776862106}, {1676, 33.68423539254731}, {1677, 33.563159682253556}, {1678, 33.46610598060115}, {1679, 33.46576401188156}, + {1680, 33.6082093792797}, {1681, 34.06829829976045}, {1682, 33.78773476628623}, {1683, 33.847057378024175}, {1684, 35.250271418221374}, {1685, 35.09767253763393}, {1686, 35.074861457363895}, {1687, 35.40140497372611}, {1688, 36.31807881704327}, {1689, 37.93153971576717}, + {1690, 37.96917757654617}, {1691, 38.22198787034048}, {1692, 38.23678368537367}, {1693, 38.20599590804662}, {1694, 38.68369317469104}, {1695, 38.79215511609461}, {1696, 38.802109502845106}, {1697, 38.753495704229714}, {1698, 38.96421874631208}, {1699, 38.90821233250077}, + {1700, 39.16182425119748}, {1701, 38.853831057337715}, {1702, 38.63611099293661}, {1703, 38.24474852988975}, {1704, 38.38469009174058}, {1705, 38.32872094403988}, {1706, 38.153737338484326}, {1707, 38.264923332806745}, {1708, 38.280563758044565}, {1709, 38.18750466726223}, + {1710, 38.56983066481024}, {1711, 39.505085657555355}, {1712, 39.69872974322229}, {1713, 40.39204144577866}, {1714, 40.48553785053598}, {1715, 41.3269897150344}, {1716, 41.609581527655735}, {1717, 41.65892696057186}, {1718, 41.837313217148804}, {1719, 41.8289668854985}, + {1720, 42.15881862704277}, {1721, 42.73710053588433}, {1722, 42.348456207492845}, {1723, 42.096157824123665}, {1724, 41.92253556951924}, {1725, 41.917137854036646}, {1726, 41.97608613143687}, {1727, 41.72574431293022}, {1728, 41.24861627279633}, {1729, 41.87472943187815}, + {1730, 41.87568170648001}, {1731, 42.17909911414902}, {1732, 42.049177661936255}, {1733, 41.97474828244962}, {1734, 41.356541003497355}, {1735, 41.22195929777421}, {1736, 41.194030410051106}, {1737, 41.392479793475935}, {1738, 41.34843546848941}, {1739, 41.34402528705148}, + {1740, 42.07661962525981}, {1741, 41.52647896910617}, {1742, 41.44561097861686}, {1743, 40.958470191837534}, {1744, 41.037829903821645}, {1745, 41.07205195864329}, {1746, 41.46906637533652}, {1747, 41.49343224957735}, {1748, 40.610182804195105}, {1749, 41.017801824810185}, + {1750, 40.79059366319508}, {1751, 40.70441966049171}, {1752, 40.70503981535662}, {1753, 39.93683574905368}, {1754, 39.752239396736506}, {1755, 39.7645522699645}, {1756, 39.55588740392734}, {1757, 39.74727156272358}, {1758, 39.9737782466988}, {1759, 40.39693575020464}, + {1760, 40.216674372755165}, {1761, 40.25136763580497}, {1762, 39.38556407545778}, {1763, 39.26119592497736}, {1764, 39.16195286452751}, {1765, 38.86948923485995}, {1766, 39.61600974311452}, {1767, 39.39522112770508}, {1768, 39.514089866690526}, {1769, 39.45799756858736}, + {1770, 38.90570527579156}, {1771, 38.80478381534035}, {1772, 38.83322167157176}, {1773, 39.64297039815119}, {1774, 40.33223301684341}, {1775, 40.41694728495109}, {1776, 40.06020383799442}, {1777, 40.07361881200165}, {1778, 39.77916448556436}, {1779, 39.96893684870016}, + {1780, 40.57894081959218}, {1781, 39.89797496618613}, {1782, 39.35261558123242}, {1783, 39.36008989987693}, {1784, 38.0512977550727}, {1785, 38.264000581925124}, {1786, 38.26169623365976}, {1787, 39.10252385765842}, {1788, 39.27088966509622}, {1789, 39.15216453676914}, + {1790, 39.41049327702329}, {1791, 39.39907719439436}, {1792, 38.82328669157878}, {1793, 39.35804609496024}, {1794, 39.52870877903105}, {1795, 40.562208982859616}, {1796, 41.015089776340226}, {1797, 40.87083252184389}, {1798, 40.53994999420047}, {1799, 41.02575733506004}, + {1800, 41.17619373256442}, {1801, 41.19685023564847}, {1802, 40.98189573065346}, {1803, 41.159763360041595}, {1804, 41.59421552557119}, {1805, 41.20806640088285}, {1806, 40.414699831836835}, {1807, 39.92415966374783}, {1808, 39.62606546104271}, {1809, 39.857180263975444}, + {1810, 39.59080107893363}, {1811, 39.6974862276464}, {1812, 39.442229052918876}, {1813, 39.30199180817845}, {1814, 39.48025720867229}, {1815, 39.5287187420122}, {1816, 38.853563440015805}, {1817, 38.77749075357144}, {1818, 39.12960232658636}, {1819, 39.07838534119988}, + {1820, 39.149292277699324}, {1821, 39.00774814154133}, {1822, 38.63589451271645}, {1823, 38.95501568270912}, {1824, 38.4982817921926}, {1825, 39.10088500152322}, {1826, 39.59621160868451}, {1827, 39.45833354062081}, {1828, 39.59936421761144}, {1829, 39.57053924615646}, + {1830, 39.55660528183884}, {1831, 39.71569674116191}, {1832, 39.71328630121623}, {1833, 39.5500745849456}, {1834, 39.52081122302288}, {1835, 38.74032920127892}, {1836, 39.19864530117539}, {1837, 39.06947571924525}, {1838, 39.34422885689023}, {1839, 39.6353362225693}, + {1840, 40.71459696346432}, {1841, 40.66496952468566}, {1842, 41.006778236777826}, {1843, 40.98904213311685}, {1844, 40.108733704463795}, {1845, 39.422159369592656}, {1846, 39.23842350070322}, {1847, 37.531170184849564}, {1848, 38.05353522108415}, {1849, 38.31111320664359}, + {1850, 38.64900610422957}, {1851, 38.686484249347565}, {1852, 38.68434197825875}, {1853, 38.72332458907287}, {1854, 38.81119764084882}, {1855, 38.46493433585336}, {1856, 38.51556317974943}, {1857, 38.764010535750494}, {1858, 38.55294929767745}, {1859, 38.078366113942565}, + {1860, 38.42988961361437}, {1861, 39.047014068329304}, {1862, 39.532635843081685}, {1863, 39.10571993423328}, {1864, 39.60131420080908}, {1865, 38.81680138357229}, {1866, 39.25082176080607}, {1867, 38.15677220409511}, {1868, 38.62555210254061}, {1869, 38.47132788365266}, + {1870, 38.487714681626166}, {1871, 38.39888830728324}, {1872, 37.54975105530615}, {1873, 36.620496899833185}, {1874, 36.76592939311881}, {1875, 36.77417927197788}, {1876, 36.78457313190018}, {1877, 36.726556157401774}, {1878, 37.19484339680942}, {1879, 37.12075085414386}, + {1880, 36.977258419880144}, {1881, 36.91078025788851}, {1882, 37.02130072401423}, {1883, 36.55535121811925}, {1884, 35.989467257305485}, {1885, 35.963032268137205}, {1886, 36.010752822606925}, {1887, 36.02787704722448}, {1888, 36.49326570398064}, {1889, 36.23934542841747}, + {1890, 35.87776015328743}, {1891, 35.09797528518658}, {1892, 35.077065290325194}, {1893, 35.311414001953885}, {1894, 34.57305509209034}, {1895, 34.6842074805997}, {1896, 34.80183031401632}, {1897, 35.47101264307737}, {1898, 35.17048394585489}, {1899, 35.089175368832215}, + {1900, 35.22509160122155}, {1901, 35.01261679236839}, {1902, 35.20621635881338}, {1903, 35.37294264170206}, {1904, 34.459027714667855}, {1905, 35.092154761591416}, {1906, 34.8517351148361}, {1907, 34.86550272062247}, {1908, 35.125600870401854}, {1909, 34.83446288221645}, + {1910, 34.78242112077965}, {1911, 34.31203602602105}, {1912, 34.279540168694986}, {1913, 35.1096803630126}, {1914, 34.514233849300744}, {1915, 34.53539867779291}, {1916, 34.279414052434674}, {1917, 34.36455267465703}, {1918, 34.10174900420402}, {1919, 34.41175163800032}, + {1920, 34.291999205288036}, {1921, 34.19594013042914}, {1922, 34.18838364564101}, {1923, 36.07392483120914}, {1924, 36.05098431555323}, {1925, 36.76095964181586}, {1926, 37.82863560027054}, {1927, 37.52980493319014}, {1928, 37.53527503569141}, {1929, 37.512668716830554}, + {1930, 37.11106227609642}, {1931, 37.349978643969806}, {1932, 37.20979207398837}, {1933, 37.29059943130983}, {1934, 37.12675552575493}, {1935, 37.34021255585861}, {1936, 37.33863802694112}, {1937, 36.93793225844994}, {1938, 36.96147855185145}, {1939, 37.079544775050316}, + {1940, 36.96800493367608}, {1941, 36.62130515314802}, {1942, 36.64654224799271}, {1943, 36.564393168165026}, {1944, 36.76798510268848}, {1945, 36.34484500467934}, {1946, 35.77622429269998}, {1947, 36.28861461359311}, {1948, 35.62308019530623}, {1949, 35.44131748549641}, + {1950, 34.85392909768805}, {1951, 34.55133668610086}, {1952, 35.092025906373635}, {1953, 34.768122961192404}, {1954, 35.02372314142307}, {1955, 35.11384033534372}, {1956, 35.09859916781124}, {1957, 35.63073140705392}, {1958, 35.47556248383545}, {1959, 34.87826460124942}, + {1960, 34.87959003890974}, {1961, 34.025365090862095}, {1962, 33.804380004654575}, {1963, 32.78020946544929}, {1964, 32.75918431846117}, {1965, 32.75851314921747}, {1966, 32.93198375479733}, {1967, 35.25828074460553}, {1968, 34.28240693566641}, {1969, 34.339864432373425}, + {1970, 34.12397376413126}, {1971, 34.79011341612875}, {1972, 34.64686103755323}, {1973, 34.599313028973235}, {1974, 34.53242934134927}, {1975, 34.606548457455425}, {1976, 34.83272344102554}, {1977, 34.49913157098365}, {1978, 34.70437557256804}, {1979, 35.19565588926615}, + {1980, 35.185492244262484}, {1981, 33.550787666460536}, {1982, 34.02484580154227}, {1983, 34.15044793215261}, {1984, 34.543313702949256}, {1985, 34.49383893491851}, {1986, 34.69468439935647}, {1987, 34.36140916945972}, {1988, 33.199535764838636}, {1989, 33.54089185371181}, + {1990, 33.25543526295492}, {1991, 33.126250139798344}, {1992, 32.1006804695537}, {1993, 32.11793625750763}, {1994, 31.547141310894098}, {1995, 31.358848538734513}, {1996, 30.96324192071878}, {1997, 31.081433577858583}, {1998, 31.004611604459708}, {1999, 30.86284767543519}, + {2000, 30.105433307161203}, {2001, 29.32024170286219}, {2002, 29.319658236582814}, {2003, 29.758958255855212}, {2004, 30.117663632341145}, {2005, 29.548890824528907}, {2006, 29.35044585015188}, {2007, 29.38762759040267}, {2008, 29.391950686507506}, {2009, 30.353380249710128}, + {2010, 29.46899984490753}, {2011, 29.33586115695675}, {2012, 29.36114381531963}, {2013, 29.256022938524392}, {2014, 29.207874497314503}, {2015, 29.266115045795534}, {2016, 29.11037246533442}, {2017, 29.98833975350206}, {2018, 29.827356428113166}, {2019, 29.196369679790173}, + {2020, 29.09133736287457}, {2021, 29.35398305475309}, {2022, 29.54604416792868}, {2023, 28.36216318117799}, {2024, 28.172545798424313}, {2025, 29.167896346989487}, {2026, 29.134747648857985}, {2027, 28.0133642954549}, {2028, 27.97694686027503}, {2029, 28.374194072568063}, + {2030, 28.37095153088029}, {2031, 28.388216858848796}, {2032, 27.992329533869835}, {2033, 27.367870148995426}, {2034, 27.251995482961384}, {2035, 27.746729670664767}, {2036, 27.679330290354688}, {2037, 28.028729073068334}, {2038, 28.028705243794963}, {2039, 28.350786367009555}, + {2040, 28.352026432670012}, {2041, 28.225867429505602}, {2042, 28.211139910084785}, {2043, 28.126532263542863}, {2044, 28.0111605756859}, {2045, 28.037986830307158}, {2046, 27.771048205304385}, {2047, 28.101282898505325}, {2048, 27.956754355558264}, {2049, 28.898451768426284}, + {2050, 28.33803560429799}, {2051, 28.51219696371789}, {2052, 28.687032331939648}, {2053, 28.519256261795114}, {2054, 28.534411822889545}, {2055, 28.475878463300383}, {2056, 28.425823720753847}, {2057, 28.531458363352172}, {2058, 28.371986321151727}, {2059, 27.281010674258564}, + {2060, 27.285423846392753}, {2061, 26.982970399775176}, {2062, 26.779917891194174}, {2063, 27.334018434853117}, {2064, 27.261844645356348}, {2065, 27.08643958372844}, {2066, 27.399456700893154}, {2067, 27.35064731231653}, {2068, 27.27964623953228}, {2069, 27.594268915970815}, + {2070, 27.529530803142432}, {2071, 27.57032488737316}, {2072, 27.4481123592095}, {2073, 26.840190037017013}, {2074, 25.848284039508844}, {2075, 27.842568040851305}, {2076, 28.053732543316965}, {2077, 28.626766043264496}, {2078, 28.147220846160913}, {2079, 28.116929624170805}, + {2080, 28.006707726319842}, {2081, 28.30104291864807}, {2082, 28.33894272086177}, {2083, 28.44881078446907}, {2084, 28.407665090419357}, {2085, 29.120266868128827}, {2086, 29.3317588176526}, {2087, 29.665040474972592}, {2088, 29.513552545038365}, {2089, 29.692597123031344}, + {2090, 28.981378296214995}, {2091, 29.09551924330645}, {2092, 29.103036068928503}, {2093, 29.546315663547908}, {2094, 29.957633494290146}, {2095, 29.904489752089027}, {2096, 29.905243264397946}, {2097, 29.046115152161175}, {2098, 29.044952829872138}, {2099, 29.266240445467794}, + {2100, 29.59245009354912}, {2101, 29.711595098413557}, {2102, 30.01909364831503}, {2103, 30.02844709237877}, {2104, 29.95602684268363}, {2105, 29.966649831798698}, {2106, 29.870132199150568}, {2107, 30.054469107153587}, {2108, 30.00047327238945}, {2109, 29.56478041606059}, + {2110, 29.90146475387948}, {2111, 29.622907648647697}, {2112, 30.309630232664347}, {2113, 29.964474309001496}, {2114, 29.49100419801057}, {2115, 29.961421989896223}, {2116, 30.346134915473066}, {2117, 30.604426349400814}, {2118, 31.112159919665434}, {2119, 31.422724201504558}, + {2120, 31.585365400843976}, {2121, 31.996984724582912}, {2122, 32.01819220164321}, {2123, 29.835521461379106}, {2124, 30.096046165850005}, {2125, 30.103782504040677}, {2126, 30.053631787942646}, {2127, 29.906750519100672}, {2128, 29.906138940448397}, {2129, 29.852649788445774}, + {2130, 30.207615016068207}, {2131, 30.820165055355577}, {2132, 30.66806843164003}, {2133, 29.933937853455785}, {2134, 29.836930249713454}, {2135, 29.643987595361455}, {2136, 30.19074784312923}, {2137, 30.22542736675569}, {2138, 29.853735660522815}, {2139, 29.84557151500528}, + {2140, 29.839193019325773}, {2141, 31.156302724258683}, {2142, 31.25546101123808}, {2143, 31.108453918938817}, {2144, 30.666860514055692}, {2145, 30.645333186456657}, {2146, 30.170151295689198}, {2147, 30.295207809019463}, {2148, 30.8518824742632}, {2149, 31.113576998851187}, + {2150, 31.31604954619179}, {2151, 31.312585749752536}, {2152, 30.864704949670024}, {2153, 30.89452281143898}, {2154, 31.23342385585233}, {2155, 31.404784721717572}, {2156, 31.18805814717659}, {2157, 31.837154644201334}, {2158, 31.806849694846097}, {2159, 31.730917911346975}, + {2160, 31.640559706015285}, {2161, 31.04070924433706}, {2162, 29.808188744723488}, {2163, 30.56052952306829}, {2164, 30.606670887563382}, {2165, 30.40372997396476}, {2166, 30.720199473446222}, {2167, 30.76154769857017}, {2168, 30.970925562360136}, {2169, 31.019650519858754}, + {2170, 31.229380140618062}, {2171, 31.388864514313752}, {2172, 31.354301610850698}, {2173, 30.99276274081928}, {2174, 31.915179561018494}, {2175, 31.918576205835393}, {2176, 33.159103841393794}, {2177, 33.29787809852651}, {2178, 33.01252169346593}, {2179, 33.02073471727393}, + {2180, 32.99606635802466}, {2181, 33.00728410289347}, {2182, 34.05607738376311}, {2183, 33.15858774223605}, {2184, 33.43289159046996}, {2185, 33.4955768254167}, {2186, 33.49850941243009}, {2187, 33.6849166584474}, {2188, 33.94841513146962}, {2189, 33.30304766805812}, + {2190, 33.22108394356688}, {2191, 32.69882923499378}, {2192, 32.427937313925405}, {2193, 32.18858592278298}, {2194, 32.32103253993895}, {2195, 32.71485240989556}, {2196, 33.33520209595963}, {2197, 33.263196044170755}, {2198, 32.710035724800086}, {2199, 33.15410338967067}, + {2200, 33.49162431986145}, {2201, 33.710578425275074}, {2202, 33.32543853826808}, {2203, 33.25527764293191}, {2204, 33.46839755330703}, {2205, 33.683203814839125}, {2206, 33.73810431818219}, {2207, 33.26839137241745}, {2208, 32.870015366666216}, {2209, 33.00655626613422}, + {2210, 33.461231149892406}, {2211, 33.89423423998636}, {2212, 33.50309147431082}, {2213, 33.579287468108}, {2214, 33.66791505848393}, {2215, 34.38967435369897}, {2216, 36.28181224897229}, {2217, 36.88357351027984}, {2218, 36.84878280662571}, {2219, 37.47217008391058}, + {2220, 37.84817431115949}, {2221, 37.90417263033801}, {2222, 37.83987872444627}, {2223, 37.32997096688406}, {2224, 37.4853701637283}, {2225, 38.101980955554055}, {2226, 37.634879540380766}, {2227, 37.458475294951995}, {2228, 37.24084436632412}, {2229, 38.45249609002218}, + {2230, 37.85699416217567}, {2231, 37.35638525020401}, {2232, 37.34244352498811}, {2233, 37.61816197522845}, {2234, 37.16649173849021}, {2235, 37.47322953068236}, {2236, 37.2333430674181}, {2237, 37.14977263499141}, {2238, 36.93966508491671}, {2239, 37.069212698488194}, + {2240, 37.13031285412952}, {2241, 37.12637993171949}, {2242, 36.91777324959473}, {2243, 36.89130918063377}, {2244, 36.83287503871216}, {2245, 37.20202075141188}, {2246, 36.83524257279788}, {2247, 36.830944738890715}, {2248, 36.98000618572528}, {2249, 37.42482479721468}, + {2250, 37.63036405693426}, {2251, 36.667366298537495}, {2252, 36.752893008045845}, {2253, 36.353792852732994}, {2254, 36.68479700023098}, {2255, 36.69141503905663}, {2256, 36.87350479821814}, {2257, 36.165233755112176}, {2258, 35.733553198879285}, {2259, 35.48805636280678}, + {2260, 35.620206059835176}, {2261, 35.633174461262}, {2262, 35.8437425347841}, {2263, 35.9725442605288}, {2264, 35.30481451302577}, {2265, 34.799856216789664}, {2266, 35.20793503084299}, {2267, 35.15095956450417}, {2268, 35.52063000497536}, {2269, 35.7847373835048}, + {2270, 35.4149729701676}, {2271, 35.20790810237125}, {2272, 35.015963290895314}, {2273, 35.152141159882405}, {2274, 34.7677733076443}, {2275, 34.48805775061282}, {2276, 34.48754941675096}, {2277, 34.31241590281594}, {2278, 34.39912341428924}, {2279, 34.214813920779555}, + {2280, 34.28073309875244}, {2281, 33.48991097769715}, {2282, 33.48007351832354}, {2283, 34.358913453235225}, {2284, 34.81847331063591}, {2285, 35.01545093767003}, {2286, 35.522962489940845}, {2287, 35.4137445712856}, {2288, 36.11012685056512}, {2289, 36.22211408792958}, + {2290, 36.18967456479244}, {2291, 36.19988717533946}, {2292, 36.82497371024283}, {2293, 36.828687017747605}, {2294, 36.81449738807559}, {2295, 36.83820772921837}, {2296, 37.24345441577025}, {2297, 36.92597317758977}, {2298, 35.8576345692627}, {2299, 35.88555065526424}, + {2300, 36.06990172210221}, {2301, 35.94117611192266}, {2302, 35.917302525035176}, {2303, 35.67840840680457}, {2304, 36.24622015952215}, {2305, 36.3620909907333}, {2306, 36.530845130699554}, {2307, 37.10057430482785}, {2308, 36.99538603909066}, {2309, 37.317285421916054}, + {2310, 37.023853418725835}, {2311, 37.331141223084856}, {2312, 36.8562932336869}, {2313, 36.872868345401656}, {2314, 37.29544282161283}, {2315, 37.162364534385944}, {2316, 37.12578408822864}, {2317, 37.27181141630765}, {2318, 37.4312789350087}, {2319, 37.13237760566537}, + {2320, 37.11211557589226}, {2321, 36.8242524461256}, {2322, 36.99871196220295}, {2323, 36.53900862036208}, {2324, 37.02861438655951}, {2325, 36.87864302249147}, {2326, 37.35471016767267}, {2327, 37.48320749003392}, {2328, 37.90669667178226}, {2329, 38.226516715835075}, + {2330, 38.22517385982157}, {2331, 38.517597628277066}, {2332, 39.13171392966848}, {2333, 38.43519575012611}, {2334, 39.019538645173135}, {2335, 38.96816384019192}, {2336, 39.18980442092051}, {2337, 39.20626086856055}, {2338, 38.91508217017819}, {2339, 39.768877736103605}, + {2340, 39.77576200648055}, {2341, 39.70340066789778}, {2342, 39.28514875368393}, {2343, 39.45020952653042}, {2344, 39.39155936608262}, {2345, 38.98575368659195}, {2346, 39.07457749590117}, {2347, 39.315569014125366}, {2348, 39.534223631257824}, {2349, 39.46496086158911}, + {2350, 39.68920036760856}, {2351, 39.76991133023266}, {2352, 39.535528882389414}, {2353, 38.78464337191674}, {2354, 38.84802165134902}, {2355, 39.13969405342098}, {2356, 39.57233319093269}, {2357, 39.4775906546037}, {2358, 39.22345113789871}, {2359, 38.57809299449735}, + {2360, 38.57645107268238}, {2361, 38.972137312472306}, {2362, 38.48045973837729}, {2363, 38.13356592124918}, {2364, 37.58820508109955}, {2365, 37.201698739301705}, {2366, 36.94671668201051}, {2367, 37.2977080973896}, {2368, 37.56900473168416}, {2369, 38.107005730339885}, + {2370, 37.66230861685099}, {2371, 37.91179580848597}, {2372, 38.021982005474676}, {2373, 37.980026494228056}, {2374, 37.936145999397155}, {2375, 38.02211453233284}, {2376, 38.184818957491586}, {2377, 38.001562339346805}, {2378, 38.17115924132589}, {2379, 38.02933185572005}, + {2380, 39.26672886459662}, {2381, 39.438886818730374}, {2382, 39.632666785719366}, {2383, 39.69431724624071}, {2384, 38.45229246819694}, {2385, 38.92044292382193}, {2386, 38.71930950151329}, {2387, 38.743236180041634}, {2388, 38.983429309450706}, {2389, 38.97420423718335}, + {2390, 38.7215640748561}, {2391, 37.29737183692541}, {2392, 37.45124430298096}, {2393, 37.528370436924256}, {2394, 37.95222937643749}, {2395, 37.83598429862198}, {2396, 37.497742223483016}, {2397, 37.445480102504746}, {2398, 37.30971600962666}, {2399, 37.35293634455916}, + {2400, 37.53645804280457}, {2401, 37.39990233842864}, {2402, 37.35307876697158}, {2403, 37.20884959153676}, {2404, 37.37721955667315}, {2405, 38.163601842424356}, {2406, 38.1744583572895}, {2407, 38.22526886325096}, {2408, 37.74537108958047}, {2409, 37.695977476731315}, + {2410, 37.132399280518335}, {2411, 37.58951197469921}, {2412, 37.595014414031965}, {2413, 37.8352174960965}, {2414, 38.19482676835253}, {2415, 38.81615407880503}, {2416, 38.96954505622985}, {2417, 38.51014481284331}, {2418, 38.70949789629867}, {2419, 38.8330673644232}, + {2420, 38.79852936780312}, {2421, 38.36989947803435}, {2422, 38.38299952696908}, {2423, 38.37911293511124}, {2424, 38.30523674843859}, {2425, 38.2780523618985}, {2426, 38.11238736059342}, {2427, 38.146761901654045}, {2428, 38.16577426042987}, {2429, 37.21503059523459}, + {2430, 37.20194861904684}, {2431, 37.93629061523524}, {2432, 37.73740404804743}, {2433, 38.19428643465362}, {2434, 38.665082590845316}, {2435, 38.73654599191024}, {2436, 38.91621864442491}, {2437, 38.91583279385087}, {2438, 39.070111723777586}, {2439, 39.020903222278925}, + {2440, 39.11691716267563}, {2441, 38.48057387525625}, {2442, 38.53568696118676}, {2443, 38.643673021215534}, {2444, 38.260311740941205}, {2445, 38.22563157129045}, {2446, 38.02954799036356}, {2447, 38.37221447795219}, {2448, 38.39689602415647}, {2449, 38.39841000702669}, + {2450, 38.42466352375716}, {2451, 38.54182197292112}, {2452, 37.97168851025008}, {2453, 37.98072923120449}, {2454, 38.500048729444565}, {2455, 38.51204601961472}, {2456, 38.57119577782335}, {2457, 38.66797170791196}, {2458, 38.42048390776265}, {2459, 38.37374323690084}, + {2460, 37.32968095967905}, {2461, 37.08263012999697}, {2462, 37.50308137969606}, {2463, 37.74228959632586}, {2464, 37.74069036775092}, {2465, 37.17849832026422}, {2466, 36.96662265441138}, {2467, 36.94747312813148}, {2468, 37.478003899130535}, {2469, 36.78538811979787}, + {2470, 36.72166338143237}, {2471, 36.05942648742004}, {2472, 35.87090337493837}, {2473, 35.86422187859753}, {2474, 36.19415669239082}, {2475, 36.4304121266589}, {2476, 37.3789805517048}, {2477, 37.50777745489909}, {2478, 37.1957829652079}, {2479, 37.17977294040911}, + {2480, 38.216074640194954}, {2481, 39.76987261793349}, {2482, 39.710123436790155}, {2483, 39.46441746053927}, {2484, 39.887390948196604}, {2485, 40.47940528392495}, {2486, 40.96173004505429}, {2487, 40.80366573878367}, {2488, 41.54175623171586}, {2489, 41.9020390419709}, + {2490, 42.33439627394617}, {2491, 43.42510969518392}, {2492, 43.3467495490359}, {2493, 43.55870952188062}, {2494, 43.687712555042445}, {2495, 43.5164043044117}, {2496, 43.50241651823093}, {2497, 44.29101702903774}, {2498, 43.557445214897555}, {2499, 43.93564846197703}, + {2500, 43.443137649447785}, {2501, 42.20225850019654}, {2502, 41.79435089000459}, {2503, 41.76196350512668}, {2504, 42.303354344610476}, {2505, 43.10084057387829}, {2506, 43.178994474468176}, {2507, 42.428481094831376}, {2508, 42.547354773308356}, {2509, 42.76222854703309}, + {2510, 41.82251108158902}, {2511, 39.95979151189301}, {2512, 39.95821075474937}, {2513, 40.79681508224348}, {2514, 40.81708050706749}, {2515, 40.86098377024993}, {2516, 41.37066167344864}, {2517, 40.811835396416136}, {2518, 41.75210935179267}, {2519, 42.081642066666376}, + {2520, 42.08290785008212}, {2521, 42.24300735871055}, {2522, 43.469996426408535}, {2523, 43.07953421381988}, {2524, 42.75861204389449}, {2525, 42.68285436365714}, {2526, 42.70557759253858}, {2527, 42.48853467614813}, {2528, 42.57942769581852}, {2529, 41.641644251781955}, + {2530, 41.995153297318495}, {2531, 41.948491392461136}, {2532, 42.10736843531852}, {2533, 42.09104450737047}, {2534, 41.844893380773094}, {2535, 41.604667841842605}, {2536, 41.664479889713284}, {2537, 41.721596977396956}, {2538, 41.03970005650016}, {2539, 41.02554512620144}, + {2540, 40.87794875769463}, {2541, 39.90420781213615}, {2542, 39.917556293894414}, {2543, 39.854472672630656}, {2544, 40.07480007115982}, {2545, 39.67254038329676}, {2546, 39.75546306998178}, {2547, 39.738281539976}, {2548, 40.68871115360799}, {2549, 40.634406366307545}, + {2550, 40.825610497929816}, {2551, 40.56134401134031}, {2552, 40.49316949904079}, {2553, 41.411980447849764}, {2554, 41.45708514923727}, {2555, 41.62699785722495}, {2556, 40.73847974293596}, {2557, 40.9036273444639}, {2558, 40.835583475220645}, {2559, 40.692726642278075}, + {2560, 40.02156508795992}, {2561, 39.59727702576626}, {2562, 39.85915504290939}, {2563, 40.25776254670131}, {2564, 38.98058165621274}, {2565, 39.19626524671679}, {2566, 40.27851337856}, {2567, 40.42198078582052}, {2568, 40.84324235005817}, {2569, 41.43936550130537}, + {2570, 41.01778584521347}, {2571, 41.71741296575474}, {2572, 41.80764781109384}, {2573, 41.86906945022645}, {2574, 41.261486612033735}, {2575, 41.05330428242297}, {2576, 40.858767555527805}, {2577, 41.20327089795783}, {2578, 41.72211067822387}, {2579, 42.24939510039889}, + {2580, 43.02032269889916}, {2581, 43.169601435406285}, {2582, 43.03240557408647}, {2583, 44.058494616358495}, {2584, 42.69632168879649}, {2585, 42.70233698941648}, {2586, 42.694697227758425}, {2587, 42.88764129390405}, {2588, 43.08913827262613}, {2589, 42.89734340644753}, + {2590, 42.541548108008676}, {2591, 42.810258239438845}, {2592, 41.86604756233666}, {2593, 41.62721508609488}, {2594, 41.42397058232155}, {2595, 41.226915898008365}, {2596, 41.362278318282556}, {2597, 40.55335045136749}, {2598, 40.53227628820762}, {2599, 40.55804572554107}, + {2600, 40.59654847444725}, {2601, 40.600759547237764}, {2602, 39.886740241510296}, {2603, 39.74441936291896}, {2604, 39.4574717226145}, {2605, 40.060274508040756}, {2606, 40.13087881643223}, {2607, 40.034347420207766}, {2608, 39.99959402815168}, {2609, 39.96436408170099}, + {2610, 40.05829543634176}, {2611, 39.36433405019379}, {2612, 39.18728728622009}, {2613, 39.24495519885113}, {2614, 39.07757095227659}, {2615, 39.274474362783934}, {2616, 39.10484953636473}, {2617, 38.82145181383544}, {2618, 39.14891944323103}, {2619, 38.38965006924832}, + {2620, 38.39353088180409}, {2621, 37.5842486836372}, {2622, 37.50572067949534}, {2623, 37.47652007277792}, {2624, 37.460762948164124}, {2625, 37.70360102326645}, {2626, 37.9284949964923}, {2627, 37.94965323046383}, {2628, 38.7335220955414}, {2629, 38.261029750354496}, + {2630, 38.32794292196558}, {2631, 38.15225856809112}, {2632, 37.90928164592873}, {2633, 38.23964767428957}, {2634, 38.52123444438283}, {2635, 38.42104882766221}, {2636, 38.96022542956203}, {2637, 39.24100815688652}, {2638, 39.08793479950826}, {2639, 39.08884782494273}, + {2640, 39.55798120001941}, {2641, 39.105610334728816}, {2642, 39.007630691250704}, {2643, 38.20582502075655}, {2644, 37.543315900244956}, {2645, 37.57555493691154}, {2646, 37.28692975360062}, {2647, 36.23295703793576}, {2648, 36.18918955965909}, {2649, 36.08967399169209}, + {2650, 35.959764744648616}, {2651, 36.19197227091526}, {2652, 36.660749297052476}, {2653, 36.64494239211856}, {2654, 36.42558293570474}, {2655, 35.6375906092785}, {2656, 35.70674524011707}, {2657, 35.57068617114742}, {2658, 35.373679501491125}, {2659, 34.672906766786355}, + {2660, 34.63171730160524}, {2661, 34.67893812694516}, {2662, 34.654847702252816}, {2663, 34.65402092594314}, {2664, 34.771606902340174}, {2665, 34.88770410184326}, {2666, 35.02613347813431}, {2667, 34.46478240438498}, {2668, 34.68568272903622}, {2669, 34.44312396898851}, + {2670, 34.21234874898513}, {2671, 34.22641653591384}, {2672, 34.128223876429104}, {2673, 34.11242598726876}, {2674, 33.96946719921404}, {2675, 33.96323823944133}, {2676, 34.10711370082857}, {2677, 33.61921918842873}, {2678, 33.86921234418534}, {2679, 34.38300717848694}, + {2680, 34.47612413434144}, {2681, 33.878358516222164}, {2682, 33.95450317313319}, {2683, 34.15914443090066}, {2684, 33.8706108159827}, {2685, 34.38200284883451}, {2686, 34.38499453227831}, {2687, 34.540332423206024}, {2688, 34.180253970681264}, {2689, 34.23917159483758}, + {2690, 33.3409032838207}, {2691, 33.31758465582364}, {2692, 33.77426222148222}, {2693, 33.98469402333139}, {2694, 35.18897714992098}, {2695, 35.27834052976643}, {2696, 35.30947212883306}, {2697, 34.697407680416966}, {2698, 35.480822427408455}, {2699, 35.56467660433882}, + {2700, 35.53420547926852}, {2701, 35.541541884085056}, {2702, 35.587731051837416}, {2703, 35.20649466374446}, {2704, 34.23840909461753}, {2705, 34.03444342209502}, {2706, 33.765366528347904}, {2707, 33.83889062651724}, {2708, 33.58019639477255}, {2709, 33.404308515206424}, + {2710, 32.5794052072048}, {2711, 32.585104546723436}, {2712, 32.266279628194816}, {2713, 31.763850208735782}, {2714, 31.82468466746643}, {2715, 31.693375409028246}, {2716, 31.713589796403312}, {2717, 31.89442805523013}, {2718, 30.539601797884956}, {2719, 30.419562355741576}, + {2720, 30.570511736331543}, {2721, 30.436298375688793}, {2722, 30.4150717914012}, {2723, 30.65434061186432}, {2724, 30.55584352842997}, {2725, 30.666073288068162}, {2726, 30.631386637749323}, {2727, 30.253653739813487}, {2728, 29.709659914781653}, {2729, 28.896044518517407}, + {2730, 28.61563883266229}, {2731, 28.70191534200063}, {2732, 29.201426696903418}, {2733, 29.55609328115069}, {2734, 28.656892877727273}, {2735, 29.07942162200021}, {2736, 29.26501321868672}, {2737, 29.646954975157882}, {2738, 29.334935013945213}, {2739, 29.050159467980038}, + {2740, 29.0197508395469}, {2741, 28.774518059870104}, {2742, 29.406230838159324}, {2743, 29.384319812035454}, {2744, 28.16252911704915}, {2745, 28.370968057426175}, {2746, 28.47565669331631}, {2747, 28.557859294540695}, {2748, 28.61247793755877}, {2749, 28.290652073805152}, + {2750, 27.951340656361044}, {2751, 28.423165549960824}, {2752, 28.44548879989753}, {2753, 28.43557149451693}, {2754, 28.041081598237184}, {2755, 27.037025918511283}, {2756, 27.037046717739166}, {2757, 26.76662330440143}, {2758, 26.699917649004856}, {2759, 26.889805976598808}, + {2760, 26.801047959108622}, {2761, 26.760134602011636}, {2762, 27.18343703683992}, {2763, 27.16859736505303}, {2764, 27.04599114188329}, {2765, 27.07137489048196}, {2766, 27.998508184180817}, {2767, 27.90008295282876}, {2768, 27.64403546500875}, {2769, 27.526129921325765}, + {2770, 27.17477572419831}, {2771, 27.341255122694978}, {2772, 27.281292839268044}, {2773, 27.457194113628123}, {2774, 27.125831819180696}, {2775, 27.153488900596923}, {2776, 26.40548276755705}, {2777, 26.20664419363637}, {2778, 25.447443897366075}, {2779, 25.436213063192547}, + {2780, 25.80691677092328}, {2781, 25.911262332850754}, {2782, 25.827467186922345}, {2783, 25.93386934563249}, {2784, 25.87141867343481}, {2785, 26.127903579867812}, {2786, 26.11594064081708}, {2787, 25.968383286170432}, {2788, 26.12819163023331}, {2789, 26.528383252253256}, + {2790, 26.63041226276307}, {2791, 26.12308929789773}, {2792, 25.824556557378628}, {2793, 25.705278327514673}, {2794, 25.533179388113364}, {2795, 25.111955575936644}, {2796, 25.19306942608654}, {2797, 25.315610017978926}, {2798, 25.54647388963821}, {2799, 25.487771689167698}, + {2800, 25.81216153045184}, {2801, 25.763987775825093}, {2802, 25.902526292298354}, {2803, 25.968428482836103}, {2804, 26.41323130119539}, {2805, 26.49882702247749}, {2806, 26.49213018635884}, {2807, 27.54190493767045}, {2808, 26.535580599916567}, {2809, 26.57432320629819}, + {2810, 26.90159184032591}, {2811, 26.587265607613453}, {2812, 26.59291668712681}, {2813, 26.423373614090597}, {2814, 27.866860910585345}, {2815, 28.207557672760228}, {2816, 28.496172416512515}, {2817, 28.42429902292854}, {2818, 28.524648108317813}, {2819, 28.611122350682358}, + {2820, 27.732969193443907}, {2821, 27.399970276602343}, {2822, 27.905344738574847}, {2823, 27.942078810801064}, {2824, 27.804282414772008}, {2825, 27.8069558632371}, {2826, 27.832235140508022}, {2827, 27.37610449016281}, {2828, 27.448768550966395}, {2829, 27.772837570338186}, + {2830, 28.168201851600365}, {2831, 28.149299964281663}, {2832, 28.131056291061356}, {2833, 27.900258674853603}, {2834, 27.900129431796973}, {2835, 27.703804331410204}, {2836, 27.233402978544067}, {2837, 28.62270120196561}, {2838, 28.753984952852}, {2839, 27.849450371851088}, + {2840, 28.607099713321475}, {2841, 28.581922133054277}, {2842, 28.44598231620244}, {2843, 28.502126322789735}, {2844, 28.33255356108348}, {2845, 28.285305697352577}, {2846, 28.09977585227804}, {2847, 27.757945405078186}, {2848, 26.392688537225578}, {2849, 26.425165673292835}, + {2850, 26.545835303452666}, {2851, 26.439538464328816}, {2852, 26.269936477378405}, {2853, 26.512109899359487}, {2854, 27.512278345593856}, {2855, 28.082715243376764}, {2856, 28.41596094710675}, {2857, 27.45862971920284}, {2858, 27.39257771938314}, {2859, 27.36345333435615}, + {2860, 27.47151777939197}, {2861, 27.950894827790314}, {2862, 27.60764930763731}, {2863, 27.60813226132976}, {2864, 27.372749995542335}, {2865, 27.521145751594496}, {2866, 27.940176424747328}, {2867, 28.35191115083003}, {2868, 27.83432934764762}, {2869, 29.15786344470242}, + {2870, 29.79498828662717}, {2871, 29.568955183772818}, {2872, 29.517280037492394}, {2873, 29.63961776908481}, {2874, 30.662190826973372}, {2875, 31.71938876522025}, {2876, 31.826598098832324}, {2877, 31.826543856914203}, {2878, 31.789643116898706}, {2879, 31.92180528432921}, + {2880, 31.80959471460479}, {2881, 31.681403018636786}, {2882, 31.191957159319717}, {2883, 30.912044142938722}, {2884, 30.34671826324825}, {2885, 29.98006193864975}, {2886, 29.472355419791427}, {2887, 29.479494293595344}, {2888, 29.752640347399335}, {2889, 30.09718091452955}, + {2890, 30.431387560542564}, {2891, 29.496695832798583}, {2892, 29.246495986640035}, {2893, 29.222427974948758}, {2894, 29.124858728303685}, {2895, 28.965028596965585}, {2896, 29.08714179149652}, {2897, 30.4283543168946}, {2898, 31.215229365152215}, {2899, 31.150712888260642}, + {2900, 30.857590831985288}, {2901, 31.460128657051456}, {2902, 31.159427314891474}, {2903, 31.165617956156684}, {2904, 31.299914014945326}, {2905, 31.11690963033505}, {2906, 30.967174302858048}, {2907, 30.995885840876113}, {2908, 31.29636960058828}, {2909, 31.434858618252605}, + {2910, 31.416675748910425}, {2911, 30.205448705736853}, {2912, 29.93565034197684}, {2913, 30.343706590216623}, {2914, 30.21957109990217}, {2915, 30.85282680317679}, {2916, 30.968361747768547}, {2917, 30.009066802972328}, {2918, 29.932154181020326}, {2919, 30.093782671075324}, + {2920, 30.404844874889584}, {2921, 30.331546653465736}, {2922, 30.147767848823545}, {2923, 29.709117723930547}, {2924, 29.3659833798085}, {2925, 28.862774209616173}, {2926, 28.51448224611682}, {2927, 28.76793928928293}, {2928, 28.673496746750068}, {2929, 28.472443770823936}, + {2930, 29.022303006465368}, {2931, 28.00637473939249}, {2932, 28.031008433611056}, {2933, 27.729183209158307}, {2934, 27.9366018038988}, {2935, 27.640659462246116}, {2936, 27.171697419833}, {2937, 26.408787788436918}, {2938, 26.28204138551267}, {2939, 25.932006217741982}, + {2940, 26.169627229499696}, {2941, 25.822279798439705}, {2942, 26.35586961220703}, {2943, 26.18132397957769}, {2944, 26.044539631613738}, {2945, 25.974095714129387}, {2946, 27.05091837318809}, {2947, 26.974693513444837}, {2948, 27.072069424664924}, {2949, 26.25090442405271}, + {2950, 26.73596169363717}, {2951, 25.905671210399422}, {2952, 25.691113922860477}, {2953, 26.299532712205195}, {2954, 24.96955683778935}, {2955, 24.888869198579197}, {2956, 24.601211234973405}, {2957, 24.034114487943505}, {2958, 24.024743202429732}, {2959, 25.019008794538035}, + {2960, 24.432773221953887}, {2961, 24.544330149848985}, {2962, 23.85023866590447}, {2963, 24.036408763069247}, {2964, 24.08876550655795}, {2965, 24.05639813443458}, {2966, 23.71814837085052}, {2967, 24.703335417148605}, {2968, 24.77916378099878}, {2969, 24.839641043240448}, + {2970, 24.853870491674087}, {2971, 25.840255119538664}, {2972, 25.818729777629176}, {2973, 25.73662194677672}, {2974, 26.312249087826}, {2975, 26.081048566805045}, {2976, 26.039636620532686}, {2977, 26.03568726819535}, {2978, 26.063007917453888}, {2979, 26.074627484746603}, + {2980, 26.040651463264155}, {2981, 26.172289495364286}, {2982, 26.62269151997738}, {2983, 26.521078122843306}, {2984, 26.48143361417722}, {2985, 26.43597116107479}, {2986, 26.479253919810066}, {2987, 26.05243489612348}, {2988, 26.065403680162337}, {2989, 26.070748034587236}, + {2990, 26.054077155633216}, {2991, 25.652831846415367}, {2992, 25.876662312969824}, {2993, 26.316574636642432}, {2994, 26.27465443874206}, {2995, 26.499843498562846}, {2996, 26.427205343025584}, {2997, 26.798259276629466}, {2998, 26.901808522372313}, {2999, 26.76079388053373}, + {3000, 26.95186823373628}, {3001, 27.034524957712406}, {3002, 26.87614970836558}, {3003, 26.088631779268834}, {3004, 26.258487166059158}, {3005, 26.263116669541553}, {3006, 26.556642147234385}, {3007, 27.1359861481772}, {3008, 27.05636721935326}, {3009, 27.782632975155}, + {3010, 28.239461231129635}, {3011, 28.171321453013107}, {3012, 27.388829894601574}, {3013, 27.765206394186475}, {3014, 27.665523688066237}, {3015, 27.652201388513607}, {3016, 27.76330159496823}, {3017, 27.652745228165806}, {3018, 29.16493980313136}, {3019, 29.760186326718667}, + {3020, 29.524598241887276}, {3021, 29.871951567195385}, {3022, 29.972022031694312}, {3023, 29.991328977556655}, {3024, 30.51114039017488}, {3025, 30.358833052283288}, {3026, 31.113437416596867}, {3027, 30.73034455732333}, {3028, 32.411088299325755}, {3029, 32.448779387658455}, + {3030, 32.40261565557047}, {3031, 33.32318930682262}, {3032, 33.53986533177098}, {3033, 34.14238204328724}, {3034, 34.324204651156585}, {3035, 34.55394232184136}, {3036, 33.80917147487097}, {3037, 33.87637554473631}, {3038, 33.52627570280564}, {3039, 32.710090552383846}, + {3040, 32.87558231122469}, {3041, 32.83723732102603}, {3042, 32.68286627184585}, {3043, 32.645629659702166}, {3044, 32.485577854400574}, {3045, 32.54464973603174}, {3046, 32.72824138287855}, {3047, 32.569049941408885}, {3048, 32.05613996926093}, {3049, 31.418578354753414}, + {3050, 31.974802932294068}, {3051, 31.517876027087098}, {3052, 31.478771719965934}, {3053, 32.33321726337759}, {3054, 32.15907422841291}, {3055, 32.39386557285096}, {3056, 32.92288104967725}, {3057, 32.58205659155766}, {3058, 33.02979168623353}, {3059, 33.104577943842884}, + {3060, 32.7508051872267}, {3061, 30.759481316657414}, {3062, 29.93908316497376}, {3063, 30.168995523464204}, {3064, 30.234201200937164}, {3065, 29.60257745821407}, {3066, 29.42426454491802}, {3067, 29.780785754603404}, {3068, 29.758857379324553}, {3069, 30.339341677481386}, + {3070, 30.17509659780736}, {3071, 30.39121521041002}, {3072, 30.39268093037097}, {3073, 30.119809979067465}, {3074, 29.755321684850596}, {3075, 30.254534421486376}, {3076, 30.27128056523592}, {3077, 30.269129730144403}, {3078, 30.268406185451322}, {3079, 30.808812815476546}, + {3080, 30.541034763482905}, {3081, 30.616071356417166}, {3082, 30.49034629358917}, {3083, 30.327451475881006}, {3084, 30.20172837588546}, {3085, 30.552615556325403}, {3086, 31.018620896589276}, {3087, 31.016826954239793}, {3088, 31.03257471371818}, {3089, 30.76388835634399}, + {3090, 30.750773848565544}, {3091, 28.90734498274122}, {3092, 28.88512555430076}, {3093, 29.235602659773928}, {3094, 29.227088109152646}, {3095, 29.265927078146902}, {3096, 29.63113527441051}, {3097, 30.325843427198762}, {3098, 30.366277557982425}, {3099, 30.81125656901041}, + {3100, 29.607530596142997}, {3101, 30.157333515983648}, {3102, 30.18646934065617}, {3103, 30.27379415294111}, {3104, 30.289365730425246}, {3105, 30.283540247033166}, {3106, 31.32710063553683}, {3107, 30.380921899830913}, {3108, 30.22467645076004}, {3109, 30.23362298145734}, + {3110, 30.61875215657774}, {3111, 30.98837784635033}, {3112, 31.03260739138466}, {3113, 30.976968690215546}, {3114, 31.79541274581999}, {3115, 32.30837337558492}, {3116, 30.958484720733267}, {3117, 31.248323567786898}, {3118, 31.17080295379335}, {3119, 31.490285494334042}, + {3120, 31.28299461589361}, {3121, 31.12746538392657}, {3122, 30.304134609467486}, {3123, 30.482783363335685}, {3124, 30.800000711544357}, {3125, 30.923176972067218}, {3126, 30.92985171567796}, {3127, 31.237871497288655}, {3128, 30.082238264022603}, {3129, 30.6089077906634}, + {3130, 31.235939986705212}, {3131, 31.14323375771582}, {3132, 31.32918643869485}, {3133, 31.47312641489513}, {3134, 31.466063254090106}, {3135, 31.425832509717182}, {3136, 32.1719990071092}, {3137, 32.41592307689316}, {3138, 32.676500312625016}, {3139, 33.35536331207698}, + {3140, 33.65570232797892}, {3141, 33.04862310178812}, {3142, 33.03181458653315}, {3143, 32.67952645100101}, {3144, 32.58771553180158}, {3145, 32.60526161774588}, {3146, 32.59144411017273}, {3147, 33.328724603980014}, {3148, 32.82736116656727}, {3149, 32.585413549083036}, + {3150, 31.909366787173354}, {3151, 31.890172235321025}, {3152, 32.01030930891559}, {3153, 31.876026787120207}, {3154, 31.177003426608458}, {3155, 31.269152245617928}, {3156, 31.321299373154623}, {3157, 31.185431050212017}, {3158, 31.33308403249633}, {3159, 31.396718431876668}, + {3160, 30.967056835014755}, {3161, 31.937036635712108}, {3162, 31.782999539504566}, {3163, 31.463231624729538}, {3164, 31.462330205674178}, {3165, 32.38149340842041}, {3166, 31.490726766628722}, {3167, 31.180181319988332}, {3168, 31.285452919047007}, {3169, 32.61725672571165}, + {3170, 32.54476600813624}, {3171, 32.867677415690224}, {3172, 32.851230576948424}, {3173, 32.447875933693695}, {3174, 32.327497251158114}, {3175, 32.54030609356738}, {3176, 33.219908261056695}, {3177, 33.05354548450645}, {3178, 32.91346861269424}, {3179, 32.972877044342646}, + {3180, 32.93786082284114}, {3181, 33.05108392047543}, {3182, 33.89537603616726}, {3183, 33.82184220941501}, {3184, 33.83547506546621}, {3185, 33.46981809068626}, {3186, 33.48014088578789}, {3187, 33.99535008164156}, {3188, 34.134687943016004}, {3189, 34.116357932617554}, + {3190, 34.3777412140907}, {3191, 34.47433796931576}, {3192, 35.329063289425584}, {3193, 36.389648583072216}, {3194, 37.152845814499535}, {3195, 37.04698767782292}, {3196, 36.924986340055234}, {3197, 36.72395148805289}, {3198, 37.099937243992414}, {3199, 37.03586955519982}, + {3200, 36.85659515898313}, {3201, 37.68701715509432}, {3202, 36.94563230667493}, {3203, 36.9801530400879}, {3204, 37.124204726164955}, {3205, 37.21769294584}, {3206, 37.36244560706185}, {3207, 37.177958656048105}, {3208, 37.17234726327357}, {3209, 37.2334453881957}, + {3210, 37.443103290909}, {3211, 37.47601907379215}, {3212, 37.492358954797}, {3213, 37.75965833545013}, {3214, 37.44614357662946}, {3215, 37.36038063096078}, {3216, 37.05950813066276}, {3217, 38.18103211360699}, {3218, 36.53683047392274}, {3219, 36.83403875076923}, + {3220, 36.799322175892655}, {3221, 35.76343601069714}, {3222, 35.732925176029}, {3223, 36.030449190392034}, {3224, 35.53535958118256}, {3225, 35.46682168640866}, {3226, 35.73775862190484}, {3227, 35.61261356902764}, {3228, 35.60401961780849}, {3229, 35.7950184685732}, + {3230, 35.680731912648454}, {3231, 35.91734464456234}, {3232, 36.215345080856956}, {3233, 36.169986316646735}, {3234, 36.19636620447901}, {3235, 35.68416087517605}, {3236, 34.782382760280335}, {3237, 34.682205263960896}, {3238, 34.6915019903415}, {3239, 34.01850844249432}, + {3240, 34.49164616505544}, {3241, 34.81821831651344}, {3242, 34.17367204635451}, {3243, 33.50257123928646}, {3244, 33.74605123682653}, {3245, 33.207146127232235}, {3246, 33.350131113450125}, {3247, 33.31108845186046}, {3248, 33.25208316113067}, {3249, 32.96860920898072}, + {3250, 32.81656955503311}, {3251, 32.596084703753874}, {3252, 32.72681616233501}, {3253, 32.989201391855794}, {3254, 32.720933976649086}, {3255, 32.76607998687764}, {3256, 32.971731400496566}, {3257, 32.978292043059476}, {3258, 32.96571499076951}, {3259, 33.54075000081999}, + {3260, 33.428210870837745}, {3261, 33.37480509177156}, {3262, 32.04401143262248}, {3263, 31.758176393318287}, {3264, 31.640748236625246}, {3265, 31.64024961699623}, {3266, 32.12120327486007}, {3267, 32.09409145385511}, {3268, 31.82108292829529}, {3269, 31.767305808090345}, + {3270, 31.61485887968538}, {3271, 31.25747957686745}, {3272, 30.343098748976058}, {3273, 30.310065993327697}, {3274, 30.08147466217843}, {3275, 29.06160851755138}, {3276, 29.649133669716544}, {3277, 29.479576201649866}, {3278, 29.53734181807828}, {3279, 29.875185928183274}, + {3280, 30.026430749893212}, {3281, 29.070477635815827}, {3282, 30.080721132734187}, {3283, 30.213354081612774}, {3284, 30.584419177806883}, {3285, 30.554197032740483}, {3286, 30.743451866698816}, {3287, 32.72273350370443}, {3288, 32.55596605650281}, {3289, 32.58240930965781}, + {3290, 32.42498977194752}, {3291, 32.61029965109377}, {3292, 32.9308941823172}, {3293, 32.89483722129089}, {3294, 32.85881557046856}, {3295, 32.726785325793635}, {3296, 33.04118883100211}, {3297, 33.83571180198559}, {3298, 33.647394009337454}, {3299, 33.520540287573574}, + {3300, 33.162168528911174}, {3301, 33.508306024446206}, {3302, 33.525507479986295}, {3303, 33.52207711917462}, {3304, 33.627821807306546}, {3305, 33.198515456466744}, {3306, 33.42607172054768}, {3307, 33.07791501061832}, {3308, 33.107977635263275}, {3309, 32.842151272849485}, + {3310, 33.05292527827272}, {3311, 32.957185834270064}, {3312, 33.06230697948977}, {3313, 33.61764522661923}, {3314, 34.3994082345058}, {3315, 34.29897488531151}, {3316, 34.21313709031122}, {3317, 34.15640542042003}, {3318, 34.189075319669456}, {3319, 34.065995348334695}, + {3320, 33.99963167652209}, {3321, 33.824584145650626}, {3322, 32.62903776472264}, {3323, 32.07197931490158}, {3324, 31.212041654609635}, {3325, 31.35529289602581}, {3326, 31.795752220664465}, {3327, 32.055153913003146}, {3328, 32.120788126976784}, {3329, 32.11630360272935}, + {3330, 32.137853597557225}, {3331, 32.09903615115943}, {3332, 32.30648496829282}, {3333, 32.07508894046564}, {3334, 32.234779372557384}, {3335, 31.75162714209973}, {3336, 31.175617555355522}, {3337, 31.132722428185286}, {3338, 30.740013555537057}, {3339, 30.80440681916967}, + {3340, 30.389128544023528}, {3341, 30.923705876216804}, {3342, 30.570681686489785}, {3343, 30.56960911154055}, {3344, 30.572573228321385}, {3345, 30.56283258873671}, {3346, 30.75768607862666}, {3347, 30.70512333660316}, {3348, 30.73660038831468}, {3349, 31.14637617533282}, + {3350, 31.20431061055002}, {3351, 31.20123522063627}, {3352, 30.97866560996787}, {3353, 31.09897337898735}, {3354, 31.592251360854707}, {3355, 31.658003377873115}, {3356, 31.749358291626148}, {3357, 31.615018574293643}, {3358, 32.13348019004665}, {3359, 32.26754114906748}, + {3360, 32.114108645990356}, {3361, 32.09349668953626}, {3362, 32.12743992074132}, {3363, 33.26855777434632}, {3364, 33.20672720628778}, {3365, 33.222054502547586}, {3366, 33.276565557113116}, {3367, 33.24616375772876}, {3368, 33.22546957275849}, {3369, 33.197145715564346}, + {3370, 32.89226183861777}, {3371, 32.63753378800023}, {3372, 32.85392625336492}, {3373, 32.654738825317295}, {3374, 33.52629206488907}, {3375, 33.551906915417874}, {3376, 33.013300850717286}, {3377, 32.45224067089088}, {3378, 33.536730399855294}, {3379, 33.382025520697695}, + {3380, 33.65246947890217}, {3381, 33.729081781626235}, {3382, 33.68914809456685}, {3383, 34.27142675843858}, {3384, 34.3384613970915}, {3385, 33.041230980966134}, {3386, 33.01273553419829}, {3387, 33.81018910800749}, {3388, 34.24585759532097}, {3389, 34.61769546449783}, + {3390, 34.83617615053483}, {3391, 35.197775455649}, {3392, 35.41194481617064}, {3393, 36.97802352361716}, {3394, 36.955361245072574}, {3395, 36.46776848088892}, {3396, 36.439365640673294}, {3397, 37.01957434647059}, {3398, 36.90031551517573}, {3399, 36.34320496911004}, + {3400, 36.27540466884174}, {3401, 35.499445889025864}, {3402, 35.438767477098544}, {3403, 35.23819008394313}, {3404, 35.19377350745659}, {3405, 35.244867145440864}, {3406, 34.786303985903714}, {3407, 34.68233052858934}, {3408, 34.68606294217492}, {3409, 35.15865659415544}, + {3410, 36.152459222278075}, {3411, 36.09225370958755}, {3412, 36.43348782533814}, {3413, 37.17740333669563}, {3414, 37.09959143159908}, {3415, 37.19283547903976}, {3416, 37.067139992874246}, {3417, 37.12896533687519}, {3418, 37.374869441323746}, {3419, 37.001095013732005}, + {3420, 35.8685171365076}, {3421, 35.7790596667844}, {3422, 35.767099214183794}, {3423, 35.66846294670802}, {3424, 36.04001746550384}, {3425, 36.0480945234249}, {3426, 36.02197200446126}, {3427, 35.60312899342081}, {3428, 35.61743213922281}, {3429, 35.481336812650724}, + {3430, 35.6265808419118}, {3431, 35.58736423372758}, {3432, 34.924601234309385}, {3433, 34.90979132407057}, {3434, 35.12660838689912}, {3435, 35.27414359176062}, {3436, 34.4572696783567}, {3437, 34.855188368960206}, {3438, 34.7694402963776}, {3439, 34.65799602194358}, + {3440, 34.63813926025997}, {3441, 34.877437006370286}, {3442, 34.15314684377819}, {3443, 34.255400076640996}, {3444, 34.396642283465276}, {3445, 33.719365607039315}, {3446, 33.641859553881055}, {3447, 33.24778570063584}, {3448, 33.03425968268789}, {3449, 31.735215667970092}, + {3450, 32.52074978898301}, {3451, 32.51001723815836}, {3452, 31.988903384703473}, {3453, 32.00635868003232}, {3454, 33.15034114204495}, {3455, 33.150708817243185}, {3456, 33.160443645038676}, {3457, 33.25145744811404}, {3458, 31.855456989765536}, {3459, 31.540691769358283}, + {3460, 31.22731634782854}, {3461, 30.109943743087637}, {3462, 30.081678413940462}, {3463, 30.071462139339655}, {3464, 31.811852478724177}, {3465, 31.7817438259946}, {3466, 31.65842780226553}, {3467, 31.645232457369303}, {3468, 31.577808697440936}, {3469, 31.965622730682476}, + {3470, 31.55851189895187}, {3471, 31.55400579073177}, {3472, 31.615309544387085}, {3473, 30.577964817545922}, {3474, 31.380957115505957}, {3475, 31.65967766234796}, {3476, 31.650632080520086}, {3477, 31.583511885018947}, {3478, 31.46866237334157}, {3479, 31.51539173204228}, + {3480, 31.683674297218584}, {3481, 31.843475655353362}, {3482, 31.833474481849517}, {3483, 31.83374497628001}, {3484, 31.297115220943716}, {3485, 31.84548063776127}, {3486, 30.76165139386353}, {3487, 30.779380693616407}, {3488, 30.560174724155342}, {3489, 31.442644349165654}, + {3490, 31.23456150289414}, {3491, 31.47065724426264}, {3492, 31.04388200251078}, {3493, 31.887381742205424}, {3494, 31.022715733669997}, {3495, 31.132724334176533}, {3496, 31.254131102633508}, {3497, 29.47070713463634}, {3498, 30.102350147175464}, {3499, 30.80347150519261}, + {3500, 31.10662961863165}, {3501, 31.069676016670755}, {3502, 31.26759264402205}, {3503, 31.469315701418683}, {3504, 31.42829010862853}, {3505, 31.20304707788888}, {3506, 30.880459790358785}, {3507, 31.732791810278144}, {3508, 31.666863184217398}, {3509, 31.0372666719452}, + {3510, 30.95557997606788}, {3511, 31.980223292918765}, {3512, 32.012017886826364}, {3513, 31.69007433474044}, {3514, 31.768581447723278}, {3515, 32.15852386773846}, {3516, 31.853809576176754}, {3517, 31.849132994909127}, {3518, 32.43271679812625}, {3519, 31.639885035846955}, + {3520, 31.634677292557605}, {3521, 31.65931351662495}, {3522, 31.64099204228493}, {3523, 31.10736248404926}, {3524, 30.95348491889584}, {3525, 30.39386488435902}, {3526, 30.082639749439995}, {3527, 31.189586327936507}, {3528, 31.43839740369831}, {3529, 31.313018471682966}, + {3530, 31.24190268841608}, {3531, 31.373635460402006}, {3532, 31.26635675448505}, {3533, 30.98293712220932}, {3534, 31.431319445797822}, {3535, 30.662258590716984}, {3536, 31.555087070316418}, {3537, 32.98014017195333}, {3538, 32.718573086874514}, {3539, 32.69374365847009}, + {3540, 32.334428458462085}, {3541, 32.79981705652948}, {3542, 32.54809889650045}, {3543, 32.25529291766223}, {3544, 31.785544957781138}, {3545, 31.689101326443183}, {3546, 31.617220851427742}, {3547, 31.549033274787455}, {3548, 32.92223632877409}, {3549, 32.8330770506699}, + {3550, 33.014591854223255}, {3551, 33.36379119283432}, {3552, 33.055540509396565}, {3553, 32.46582242202945}, {3554, 32.09927951615547}, {3555, 32.68733881264074}, {3556, 33.86320988116308}, {3557, 33.94494214738513}, {3558, 33.529412010032885}, {3559, 33.4888869543669}, + {3560, 33.38035950901418}, {3561, 33.31012171482195}, {3562, 33.48535073196302}, {3563, 33.597392261160415}, {3564, 33.4512300687611}, {3565, 32.81838924040587}, {3566, 33.41685969176084}, {3567, 33.44158744976233}, {3568, 31.864631180692385}, {3569, 31.69972472198296}, + {3570, 31.62978284463028}, {3571, 33.02109212331147}, {3572, 33.012962884618574}, {3573, 32.541542372624065}, {3574, 31.491346373021337}, {3575, 31.615350692550003}, {3576, 31.804732803514614}, {3577, 31.785970967334098}, {3578, 31.302173527640477}, {3579, 31.066768598626613}, + {3580, 30.83902872742016}, {3581, 30.647189213101132}, {3582, 30.679283971247305}, {3583, 30.635220918876687}, {3584, 30.474194541456203}, {3585, 30.323171221808224}, {3586, 30.231880521112476}, {3587, 30.14165312650818}, {3588, 30.587932180323367}, {3589, 30.515797815362596}, + {3590, 30.697580128792886}, {3591, 30.5092767324158}, {3592, 30.22933286653618}, {3593, 30.399883720364244}, {3594, 30.563655322165122}, {3595, 30.916284482811637}, {3596, 31.266903691899042}, {3597, 31.277953104128557}, {3598, 31.248343781043335}, {3599, 31.665546713305297}, + {3600, 31.780736348640186}, {3601, 32.560705734243456}, {3602, 32.58949125031766}, {3603, 32.648379121925466}, {3604, 32.648994209302636}, {3605, 32.89262541394924}, {3606, 32.386706534468246}, {3607, 33.10476073358268}, {3608, 32.127562371704016}, {3609, 32.08021758550007}, + {3610, 32.01828734139976}, {3611, 32.149616630644026}, {3612, 32.24939276030207}, {3613, 31.908269478103314}, {3614, 31.75980113913783}, {3615, 31.599177196466123}, {3616, 30.82883453465096}, {3617, 30.805848182195113}, {3618, 30.67015361507686}, {3619, 30.82551357191438}, + {3620, 31.231900275482825}, {3621, 31.39570326592559}, {3622, 31.220478731815824}, {3623, 31.191057259483284}, {3624, 31.71256738228641}, {3625, 31.767353330007293}, {3626, 31.7421965553774}, {3627, 32.169320034067475}, {3628, 32.83045534082564}, {3629, 33.09082984672529}, + {3630, 33.091418882239445}, {3631, 33.534698294315376}, {3632, 34.98860015194777}, {3633, 34.887785100600965}, {3634, 34.52612439270598}, {3635, 35.222998429400846}, {3636, 35.32696073852328}, {3637, 33.91614263793164}, {3638, 33.955178987781}, {3639, 32.98821123005988}, + {3640, 31.54612425654933}, {3641, 30.47179194130409}, {3642, 29.121801087250198}, {3643, 28.98075756334753}, {3644, 29.21120341558452}, {3645, 28.94324522035289}, {3646, 30.61040164869518}, {3647, 30.938644800763065}, {3648, 31.88964085600857}, {3649, 31.104255236657654}, + {3650, 30.80739952109552}, {3651, 30.87061979495017}, {3652, 30.50628535976767}, {3653, 30.526826937324103}, {3654, 30.39733248162421}, {3655, 30.396253710063263}, {3656, 30.500732657862326}, {3657, 30.464941464184626}, {3658, 30.397674823493038}, {3659, 31.559828803294607}, + {3660, 30.4775995110338}, {3661, 30.605373430398316}, {3662, 30.959781079195544}, {3663, 30.686376335617936}, {3664, 30.45394463567413}, {3665, 30.836569443595376}, {3666, 30.41004287663308}, {3667, 30.76197883114409}, {3668, 30.17037689073509}, {3669, 30.057913986094512}, + {3670, 30.087583283388085}, {3671, 30.21878224005264}, {3672, 30.246448076959325}, {3673, 30.58839113549538}, {3674, 31.47426930437642}, {3675, 31.010529526010682}, {3676, 31.303422051514435}, {3677, 31.003100220279816}, {3678, 30.98326209088546}, {3679, 30.69183779794675}, + {3680, 31.223367625034683}, {3681, 30.789280075755688}, {3682, 30.19829216553304}, {3683, 30.363595369442493}, {3684, 30.27935553254929}, {3685, 30.1261971935347}, {3686, 30.79863985302651}, {3687, 31.047595403218057}, {3688, 30.90228673518786}, {3689, 30.543388017289054}, + {3690, 30.704356144092372}, {3691, 30.63973229439528}, {3692, 30.68310619503808}, {3693, 30.603549759307885}, {3694, 31.374934823463768}, {3695, 31.389460746452443}, {3696, 31.904273713484496}, {3697, 31.885063882176844}, {3698, 31.820524013817312}, {3699, 31.82770668437377}, + {3700, 31.62129391692174}, {3701, 33.01039885910495}, {3702, 32.810936220934124}, {3703, 33.11606920366353}, {3704, 33.45236130557991}, {3705, 34.35768750096602}, {3706, 33.480461372240505}, {3707, 33.73411429026118}, {3708, 33.89279065017372}, {3709, 34.23881544215997}, + {3710, 34.935789977307365}, {3711, 34.77481188799792}, {3712, 34.73666168190101}, {3713, 34.049290860266446}, {3714, 34.3821020789841}, {3715, 34.6030641653466}, {3716, 34.69594103398987}, {3717, 34.17980982257179}, {3718, 34.16518765028321}, {3719, 34.23763587885022}, + {3720, 34.601596281681694}, {3721, 35.4175582873499}, {3722, 35.59749071347688}, {3723, 34.32925786634385}, {3724, 34.51637645217182}, {3725, 34.525950151725944}, {3726, 35.998069795400646}, {3727, 35.963551274884786}, {3728, 36.036689984025955}, {3729, 37.24781768842208}, + {3730, 37.277514336598095}, {3731, 37.30863902603817}, {3732, 36.79217135191674}, {3733, 36.79064349318625}, {3734, 36.001098652284306}, {3735, 35.86481428501773}, {3736, 35.94802206870304}, {3737, 37.14027447150489}, {3738, 36.258549896818224}, {3739, 36.802348037612525}, + {3740, 36.89335332794575}, {3741, 37.908095460582}, {3742, 37.71387537919735}, {3743, 37.25060233029356}, {3744, 36.911971103005506}, {3745, 36.621626787271495}, {3746, 36.78598563390066}, {3747, 36.78867687247616}, {3748, 36.846211552881314}, {3749, 36.68451178474116}, + {3750, 36.30880218835804}, {3751, 36.74490372547481}, {3752, 36.93776738041527}, {3753, 36.92794573529495}, {3754, 36.88702244570728}, {3755, 37.032752352208334}, {3756, 37.09031587985092}, {3757, 37.25861213788812}, {3758, 37.0208483030904}, {3759, 37.44627314073198}, + {3760, 36.60326002825027}, {3761, 36.5349242191796}, {3762, 36.18844767465498}, {3763, 36.58284312296601}, {3764, 36.501734860814345}, {3765, 36.047516333562704}, {3766, 35.32968089985233}, {3767, 36.28983500653037}, {3768, 35.9811194049413}, {3769, 36.10451903716811}, + {3770, 36.34687133526118}, {3771, 36.31609256803086}, {3772, 37.63818828180108}, {3773, 38.43083167710727}, {3774, 38.457277972233115}, {3775, 38.50074370521922}, {3776, 38.57905462745658}, {3777, 37.94556968927861}, {3778, 38.250174481197824}, {3779, 38.87016646551586}, + {3780, 38.96356411591443}, {3781, 38.81436757446562}, {3782, 38.70813805142539}, {3783, 38.36257955910033}, {3784, 38.334587300767666}, {3785, 38.573334533834945}, {3786, 38.706937363024785}, {3787, 38.456878484269325}, {3788, 38.59652287562196}, {3789, 38.87303482456266}, + {3790, 38.26707578973226}, {3791, 38.08710058110894}, {3792, 38.107611873182606}, {3793, 37.87348574106414}, {3794, 38.467343581667315}, {3795, 38.52284971229728}, {3796, 38.80030552742671}, {3797, 39.31444077052365}, {3798, 39.36714339199697}, {3799, 39.265510040584786}, + {3800, 39.35757249633458}, {3801, 39.23029087940918}, {3802, 40.236476990129525}, {3803, 40.40447274896678}, {3804, 39.90050059379676}, {3805, 39.82968538918735}, {3806, 39.568796564237815}, {3807, 39.66844747677395}, {3808, 39.48947001553621}, {3809, 39.07804466925928}, + {3810, 39.08468133372294}, {3811, 39.409077167178886}, {3812, 39.59388813521032}, {3813, 39.61292554954707}, {3814, 39.53006069194008}, {3815, 39.43424867493228}, {3816, 39.364325046233276}, {3817, 39.24878540639196}, {3818, 38.91030638030182}, {3819, 39.19021407549976}, + {3820, 39.52465873164222}, {3821, 39.30579054542478}, {3822, 39.066334049300906}, {3823, 38.445919893706396}, {3824, 38.82331512394734}, {3825, 38.722751358387455}, {3826, 39.18918989725993}, {3827, 39.3325423377068}, {3828, 40.07164954275041}, {3829, 40.07074505067318}, + {3830, 40.27305857026511}, {3831, 40.29841893573262}, {3832, 40.35632841195591}, {3833, 41.25954776973257}, {3834, 41.08661644599172}, {3835, 40.930860440282444}, {3836, 40.67035270986339}, {3837, 40.59686023107034}, {3838, 40.966306832072135}, {3839, 41.73206011126974}, + {3840, 41.83313880921912}, {3841, 42.710027568498774}, {3842, 43.09298354963694}, {3843, 43.97481631223127}, {3844, 43.06984802315324}, {3845, 42.88682049659426}, {3846, 42.70391340792306}, {3847, 42.67733443704977}, {3848, 42.312517547282376}, {3849, 42.68248973003122}, + {3850, 41.89851060373508}, {3851, 42.499365091434946}, {3852, 42.04285692039637}, {3853, 42.36335659371158}, {3854, 43.00859290667485}, {3855, 42.64595368210728}, {3856, 42.84344419146314}, {3857, 42.58554402244061}, {3858, 42.94977322974173}, {3859, 42.970561962196946}, + {3860, 43.09387131088587}, {3861, 43.5526747563676}, {3862, 43.59839466763858}, {3863, 43.491401190645476}, {3864, 43.890847555316626}, {3865, 43.90554980755929}, {3866, 43.29419748320046}, {3867, 43.19271817040813}, {3868, 43.15798450802592}, {3869, 44.936354699949135}, + {3870, 44.925896564304466}, {3871, 45.141313850266805}, {3872, 45.15805553382705}, {3873, 45.175146274315395}, {3874, 44.602873917897206}, {3875, 43.98425210064129}, {3876, 44.21112444021221}, {3877, 44.31207946892926}, {3878, 44.32915256192521}, {3879, 44.54003640246446}, + {3880, 44.43388783676019}, {3881, 44.377913951958845}, {3882, 44.08660155118512}, {3883, 44.084088844428344}, {3884, 44.678908282742874}, {3885, 44.69090507621899}, {3886, 44.66399823516045}, {3887, 44.70003406608551}, {3888, 44.64217161352199}, {3889, 44.598427607594914}, + {3890, 44.64183638547881}, {3891, 45.313385126126406}, {3892, 45.213056881203656}, {3893, 45.83751311620533}, {3894, 45.97798477844367}, {3895, 46.69554397624657}, {3896, 46.780476914271844}, {3897, 46.42161452367823}, {3898, 46.485265361946965}, {3899, 46.463419227463945}, + {3900, 46.40619518750473}, {3901, 46.06654509998825}, {3902, 46.57689530438109}, {3903, 46.23083598006043}, {3904, 46.574270685203786}, {3905, 46.78507045779222}, {3906, 46.762325322870765}, {3907, 47.013171115464665}, {3908, 47.35684909712627}, {3909, 47.46573772776851}, + {3910, 47.79168684198062}, {3911, 46.864811059928705}, {3912, 46.19102604217612}, {3913, 45.57512279475959}, {3914, 45.27697004686091}, {3915, 44.43233105401431}, {3916, 44.48913258511433}, {3917, 44.05739070330766}, {3918, 43.43326838667846}, {3919, 43.18101584512046}, + {3920, 43.12661207742961}, {3921, 43.20235581216143}, {3922, 43.21124585054493}, {3923, 43.54158862241011}, {3924, 43.50571782863192}, {3925, 43.711364180631804}, {3926, 43.18168675879529}, {3927, 42.89854097019134}, {3928, 42.910583730646124}, {3929, 43.18510405314944}, + {3930, 42.639952988135626}, {3931, 42.797465701366804}, {3932, 43.44286812172023}, {3933, 43.19594242280566}, {3934, 43.423279844975504}, {3935, 43.637554324106404}, {3936, 43.56664792791941}, {3937, 44.47718122890482}, {3938, 42.71893554253818}, {3939, 42.68619011219234}, + {3940, 42.764744246822005}, {3941, 42.368141211578596}, {3942, 42.72177242428145}, {3943, 42.58290514752061}, {3944, 42.500109121456575}, {3945, 43.06917028887632}, {3946, 43.1390878438651}, {3947, 43.67566420376484}, {3948, 43.32879166406192}, {3949, 43.12282803410909}, + {3950, 43.53504871953674}, {3951, 43.11341790856445}, {3952, 43.09460900193214}, {3953, 42.975278393392486}, {3954, 43.02716626297875}, {3955, 43.28841760595471}, {3956, 43.15013781306844}, {3957, 43.05005871018039}, {3958, 42.643291525484216}, {3959, 41.40330273303392}, + {3960, 41.190483201302776}, {3961, 41.18851255115125}, {3962, 41.139227053508684}, {3963, 41.262957426835335}, {3964, 41.56188105204588}, {3965, 41.29178612732812}, {3966, 42.1903083759515}, {3967, 43.24502540234582}, {3968, 43.53636550586511}, {3969, 43.2653460646287}, + {3970, 43.08283306703269}, {3971, 42.4092975371573}, {3972, 42.339747488590305}, {3973, 42.17938430330037}, {3974, 42.9302231459865}, {3975, 44.14104375702491}, {3976, 44.32654181837024}, {3977, 44.85103730540091}, {3978, 43.686022699020185}, {3979, 43.52045377054748}, + {3980, 44.066949791075196}, {3981, 43.54943797407643}, {3982, 43.55681098163001}, {3983, 43.62378293786004}, {3984, 43.27719869525927}, {3985, 43.520080202226026}, {3986, 43.48833320696304}, {3987, 43.528692784115066}, {3988, 43.39572733345793}, {3989, 43.66669909833283}, + {3990, 43.627624602729895}, {3991, 44.99085408453511}, {3992, 44.957885207190635}, {3993, 44.75608239056898}, {3994, 44.79205318911418}, {3995, 45.071567286436604}, {3996, 45.0773216187218}, {3997, 45.20628655580068}, {3998, 45.042340799331626}, {3999, 44.40459780963831}, + {4000, 44.84911779417697}, {4001, 44.84679216126694}, {4002, 44.9926860827092}, {4003, 45.332476785661704}, {4004, 45.32599258516697}, {4005, 44.97196102902279}, {4006, 44.975532895597816}, {4007, 45.03190754404627}, {4008, 43.66039558703255}, {4009, 43.82989258131152}, + {4010, 44.06126134232322}, {4011, 44.30529606181042}, {4012, 44.44658393531378}, {4013, 43.738910733073006}, {4014, 44.212806733919}, {4015, 44.10630077043948}, {4016, 43.99563053457421}, {4017, 44.095566651377375}, {4018, 43.83450825940757}, {4019, 43.84846576986777}, + {4020, 42.71505037386405}, {4021, 43.00488908696087}, {4022, 42.4995621932694}, {4023, 42.28588980382473}, {4024, 42.304954106759766}, {4025, 41.580902668274064}, {4026, 42.195928955837026}, {4027, 42.32891452704911}, {4028, 42.394421598632725}, {4029, 42.88514428175226}, + {4030, 42.88411017501612}, {4031, 42.44465234171097}, {4032, 42.423161360690465}, {4033, 42.06024809501421}, {4034, 41.46983767233033}, {4035, 41.450540092235606}, {4036, 41.551824782132655}, {4037, 41.497581890358525}, {4038, 41.74337678928221}, {4039, 42.01458088760071}, + {4040, 42.000161946828975}, {4041, 42.09809076103792}, {4042, 40.40121722063858}, {4043, 40.405347534544276}, {4044, 41.45596015966376}, {4045, 41.34524024668175}, {4046, 41.48790124860581}, {4047, 40.85226478482957}, {4048, 41.72644422704275}, {4049, 42.28054826801833}, + {4050, 42.470889350803844}, {4051, 42.58707283732388}, {4052, 43.376033472640444}, {4053, 43.273463592859194}, {4054, 42.91653543267234}, {4055, 43.68837327519839}, {4056, 44.35550220099082}, {4057, 44.303862119305684}, {4058, 44.268994539070036}, {4059, 44.12291558250871}, + {4060, 44.15997387053544}, {4061, 43.847891173179995}, {4062, 43.88605800805093}, {4063, 43.76622485923872}, {4064, 44.08049094209474}, {4065, 43.52840705131174}, {4066, 43.47588617544269}, {4067, 43.42547149366908}, {4068, 43.620164305725254}, {4069, 43.62469613986601}, + {4070, 43.35722721012197}, {4071, 43.11523304678633}, {4072, 43.21270244719499}, {4073, 42.799008504053795}, {4074, 42.579441010470354}, {4075, 43.616467334850654}, {4076, 44.21857842681485}, {4077, 44.096520800225065}, {4078, 44.076122818906754}, {4079, 43.882145702606714}, + {4080, 44.025173271395175}, {4081, 44.08675228361723}, {4082, 44.08280580012843}, {4083, 44.089090764695634}, {4084, 44.63027712418217}, {4085, 44.64257806553344}, {4086, 44.63488930186029}, {4087, 43.49185769105879}, {4088, 43.81922760361489}, {4089, 44.55770909655596}, + {4090, 44.495768552475916}, {4091, 44.70529568081817}, {4092, 44.608267360939585}, {4093, 44.32192140208638}, {4094, 44.32913149339295}, {4095, 43.92083958045917}, {4096, 43.93629699159078}, {4097, 44.42006096186749}, {4098, 44.436560493979954}, {4099, 45.03381928301654}, + {4100, 44.89092252034728}, {4101, 45.228836061954944}, {4102, 44.96315616449893}, {4103, 45.317252160358144}, {4104, 45.3699223441705}, {4105, 44.99809335119828}, {4106, 45.31350010825937}, {4107, 45.28642508634763}, {4108, 45.23753191697613}, {4109, 45.55004475954644}, + {4110, 46.058774601853955}, {4111, 45.83557796236126}, {4112, 45.67893880941216}, {4113, 45.58586429125116}, {4114, 45.483336441906616}, {4115, 44.95324979965775}, {4116, 44.766511487040916}, {4117, 44.863928712529}, {4118, 45.521105859507294}, {4119, 45.597789271022776}, + {4120, 45.548218639658785}, {4121, 45.5671964393491}, {4122, 45.66840481231619}, {4123, 45.421280141591325}, {4124, 45.42131956848279}, {4125, 46.48565429648017}, {4126, 46.691978861656345}, {4127, 46.7203310903226}, {4128, 47.83269632839022}, {4129, 47.818072148512215}, + {4130, 47.904859885869236}, {4131, 47.7041273974865}, {4132, 47.69720727111671}, {4133, 48.00314297086582}, {4134, 46.56039022043427}, {4135, 44.90413846099235}, {4136, 45.52514774325245}, {4137, 46.058621957431804}, {4138, 46.12588682589276}, {4139, 46.58227592497253}, + {4140, 46.17055554294809}, {4141, 46.80784968189301}, {4142, 46.49508518393976}, {4143, 46.500309270278464}, {4144, 46.414835172420304}, {4145, 46.304075582190514}, {4146, 46.22819939150845}, {4147, 45.85490163160914}, {4148, 46.16449836318747}, {4149, 46.36607159615689}, + {4150, 45.33765696135226}, {4151, 45.51784949111958}, {4152, 45.27464232646123}, {4153, 45.64354971534378}, {4154, 45.57676214832657}, {4155, 45.56569880057714}, {4156, 45.645824829272414}, {4157, 45.6060530616635}, {4158, 45.78376908108006}, {4159, 45.92037288384215}, + {4160, 45.93018712100647}, {4161, 45.04435615370153}, {4162, 43.97591673735583}, {4163, 43.60804290415924}, {4164, 43.25559660341626}, {4165, 43.30560502979643}, {4166, 43.32162929680021}, {4167, 43.220797929589125}, {4168, 43.88958904031423}, {4169, 43.92613927037596}, + {4170, 43.28570468545945}, {4171, 43.33758546950846}, {4172, 43.38200781871171}, {4173, 43.37439151387566}, {4174, 43.62114145159183}, {4175, 42.583155075251156}, {4176, 41.79257202974383}, {4177, 41.5301578370286}, {4178, 40.72846655122725}, {4179, 40.9074303438794}, + {4180, 41.486828841681415}, {4181, 41.43693955839994}, {4182, 41.25469470037508}, {4183, 41.3152306690082}, {4184, 42.05756123320117}, {4185, 41.976765116312805}, {4186, 42.55329890940964}, {4187, 42.32763877115726}, {4188, 42.331621409998704}, {4189, 41.678572270110294}, + {4190, 41.78167622000538}, {4191, 41.57841592927911}, {4192, 41.41582587790593}, {4193, 40.829011801535664}, {4194, 42.57985084363515}, {4195, 42.4375122885433}, {4196, 42.32575825185156}, {4197, 42.800600639022605}, {4198, 42.07267000939594}, {4199, 41.991890779837725}, + {4200, 42.01125834670128}, {4201, 42.01783872316503}, {4202, 41.99633327331731}, {4203, 42.2264225459579}, {4204, 42.41931446868698}, {4205, 42.5104434476498}, {4206, 42.51820041440961}, {4207, 43.10274662201198}, {4208, 43.03752438898408}, {4209, 43.19323932573483}, + {4210, 43.15253357742187}, {4211, 42.158082291323616}, {4212, 42.18530485919557}, {4213, 42.179672734709584}, {4214, 42.08370843694731}, {4215, 42.062223021469975}, {4216, 41.67838942952538}, {4217, 41.49506764318014}, {4218, 41.59881716888323}, {4219, 41.926024236261355}, + {4220, 42.11947567033156}, {4221, 42.26187787539394}, {4222, 42.65015253639434}, {4223, 42.441042770154844}, {4224, 43.08690587956467}, {4225, 44.39310986424937}, {4226, 44.94461257206579}, {4227, 44.78804776055559}, {4228, 45.53797859707034}, {4229, 45.55187883708798}, + {4230, 45.88537683206737}, {4231, 45.825185517872704}, {4232, 46.07574947104996}, {4233, 46.060884716412254}, {4234, 45.76160043499644}, {4235, 45.76931072837809}, {4236, 45.51959493255436}, {4237, 45.554677642229564}, {4238, 45.52426093953901}, {4239, 46.109945660686115}, + {4240, 45.99257414377729}, {4241, 46.07894808606258}, {4242, 44.286086348874676}, {4243, 44.4865400708932}, {4244, 44.234913797823154}, {4245, 44.16033974830902}, {4246, 43.44966049381201}, {4247, 43.08447818539749}, {4248, 43.205930480034986}, {4249, 43.27174171858989}, + {4250, 43.37829207780095}, {4251, 43.10867398251995}, {4252, 43.143056746791466}, {4253, 43.184584390833216}, {4254, 43.2384965784782}, {4255, 43.64269672822312}, {4256, 43.95024182576365}, {4257, 44.34389839798999}, {4258, 44.779096753781424}, {4259, 44.81584343357247}, + {4260, 44.862897971461486}, {4261, 44.809095330679426}, {4262, 44.82258689695792}, {4263, 44.94257959545199}, {4264, 44.70649326700459}, {4265, 44.83403801956116}, {4266, 44.69375741776056}, {4267, 44.246909347585806}, {4268, 44.31131992100262}, {4269, 44.30870396354328}, + {4270, 45.277184693727}, {4271, 45.325741450057016}, {4272, 45.35787945327525}, {4273, 46.26696427252453}, {4274, 46.3349833461467}, {4275, 46.262367003647526}, {4276, 45.88415082220467}, {4277, 45.69088772324733}, {4278, 46.30744653697344}, {4279, 46.109976226034966}, + {4280, 46.00659771921764}, {4281, 45.98327288475588}, {4282, 45.93683158647156}, {4283, 45.827017445641175}, {4284, 44.898780786668084}, {4285, 44.857946447901604}, {4286, 44.78129029650326}, {4287, 44.460979780768575}, {4288, 43.781370713844545}, {4289, 43.80346385692286}, + {4290, 43.70136533034899}, {4291, 43.59605587452415}, {4292, 46.60711089281704}, {4293, 43.20999116221206}, {4294, 43.61705810567689}, {4295, 43.10115465773163}, {4296, 43.384267987646375}, {4297, 43.335817467294284}, {4298, 43.450535768186434}, {4299, 43.27624559738883}, + {4300, 43.74264705432361}, {4301, 43.80047106789878}, {4302, 43.1793620620206}, {4303, 44.06844891439754}, {4304, 44.122958015118876}, {4305, 44.16868478059633}, {4306, 43.763355009011846}, {4307, 44.24245012746741}, {4308, 43.6667108237416}, {4309, 43.44755931800053}, + {4310, 43.8388540119679}, {4311, 43.84684532813053}, {4312, 43.891606967283025}, {4313, 43.61609318450177}, {4314, 43.50391313644561}, {4315, 43.4370124294098}, {4316, 43.320512498718685}, {4317, 43.43317762888966}, {4318, 43.441565229865844}, {4319, 43.89166237920707}, + {4320, 43.79297096639801}, {4321, 42.86616397750371}, {4322, 42.492817070603124}, {4323, 42.463597734904255}, {4324, 42.099203979340935}, {4325, 43.148352379283736}, {4326, 43.14393983822119}, {4327, 43.802920465762334}, {4328, 43.787438894126545}, {4329, 44.8390211407645}, + {4330, 43.9210831019679}, {4331, 44.160162794499165}, {4332, 44.603706152703076}, {4333, 44.6029921449433}, {4334, 45.71896157142412}, {4335, 45.7186083077643}, {4336, 46.132237714004425}, {4337, 45.29810811659616}, {4338, 45.44893704926406}, {4339, 44.56756839730384}, + {4340, 44.84857422415435}, {4341, 44.59101148982434}, {4342, 45.832174813792975}, {4343, 46.24568332480992}, {4344, 46.27972490525816}, {4345, 46.220037760327614}, {4346, 46.26785697686892}, {4347, 45.91029124256099}, {4348, 46.01333033455775}, {4349, 45.55640134102827}, + {4350, 45.07332328943664}, {4351, 44.05034495051119}, {4352, 44.37631641742659}, {4353, 44.47510379397457}, {4354, 44.481536987385375}, {4355, 45.09815452419815}, {4356, 44.23545269297595}, {4357, 44.35190888940349}, {4358, 44.44376512206195}, {4359, 44.35377582256026}, + {4360, 44.182956840667266}, {4361, 44.4297999192324}, {4362, 44.13473879958124}, {4363, 44.055995402850435}, {4364, 44.005471335161786}, {4365, 43.25758809794344}, {4366, 43.817908095986404}, {4367, 43.822056761477334}, {4368, 44.14477410502386}, {4369, 43.797486505074176}, + {4370, 43.79800963408963}, {4371, 42.8796997482786}, {4372, 42.88556424491737}, {4373, 42.77727527309622}, {4374, 42.35778308150795}, {4375, 42.16975466552083}, {4376, 41.93028553518078}, {4377, 42.09563715627173}, {4378, 41.17204610689614}, {4379, 41.78572427771177}, + {4380, 41.92150205271964}, {4381, 42.09741338103042}, {4382, 43.69386481163199}, {4383, 42.96742035210106}, {4384, 42.343580110976134}, {4385, 42.22399465949221}, {4386, 42.6091386627358}, {4387, 42.568896188906415}, {4388, 42.64410302487118}, {4389, 41.95886131685545}, + {4390, 40.80681239206415}, {4391, 39.38331357790885}, {4392, 39.78427361503698}, {4393, 39.93810078981509}, {4394, 40.13352169285055}, {4395, 40.21778170710591}, {4396, 40.16970625785058}, {4397, 40.49257098503543}, {4398, 39.526963187820655}, {4399, 39.39632170400282}, + {4400, 38.65146414419247}, {4401, 38.69622195198017}, {4402, 38.82824084957831}, {4403, 38.35894467631948}, {4404, 38.374288310893164}, {4405, 38.30201987865161}, {4406, 39.040498102853704}, {4407, 38.739032917012814}, {4408, 38.21513757304971}, {4409, 37.79412671670286}, + {4410, 37.572863184633725}, {4411, 36.71659398164643}, {4412, 36.9686014580944}, {4413, 36.61396796753844}, {4414, 36.21992527315708}, {4415, 36.544907354858005}, {4416, 36.94702029184928}, {4417, 37.040712123103496}, {4418, 37.86983316239789}, {4419, 37.9133259993023}, + {4420, 37.740462632345725}, {4421, 37.80334582875091}, {4422, 38.06669175124991}, {4423, 38.82569943174218}, {4424, 38.79645166307203}, {4425, 38.138183023387136}, {4426, 37.343389010610906}, {4427, 37.06893203484559}, {4428, 36.974042644355606}, {4429, 37.10878943344164}, + {4430, 36.92199614208741}, {4431, 36.31927104688222}, {4432, 36.318646750338225}, {4433, 35.93959392178788}, {4434, 35.957928276775064}, {4435, 36.57803220920125}, {4436, 36.48729877130802}, {4437, 36.08934258703699}, {4438, 35.92075815637091}, {4439, 35.97790853205448}, + {4440, 37.10428719217352}, {4441, 37.34763949569804}, {4442, 37.54555366814172}, {4443, 38.21664941647024}, {4444, 38.15910673215726}, {4445, 38.18389302943226}, {4446, 38.15032771707228}, {4447, 38.257644304614324}, {4448, 37.89935320642902}, {4449, 37.92806972052069}, + {4450, 38.44876254366418}, {4451, 38.39207298891007}, {4452, 37.688682446222174}, {4453, 37.5187537924215}, {4454, 37.51914139540737}, {4455, 37.09540108270616}, {4456, 37.73546306496036}, {4457, 37.81369411954859}, {4458, 36.83626164610845}, {4459, 36.93994071270752}, + {4460, 36.95018579100306}, {4461, 37.26709009753631}, {4462, 37.15237030254232}, {4463, 37.049394970609185}, {4464, 37.7162630971784}, {4465, 37.778299282663106}, {4466, 38.730315146387746}, {4467, 38.7451015147541}, {4468, 38.61705719495839}, {4469, 37.872865556491874}, + {4470, 37.89246439250997}, {4471, 38.16788956159939}, {4472, 37.860151832562025}, {4473, 37.73726372456453}, {4474, 37.69142767992998}, {4475, 37.63232006550031}, {4476, 37.423877674083585}, {4477, 37.4556569056629}, {4478, 37.96421499308509}, {4479, 37.81211863240172}, + {4480, 38.05702647753458}, {4481, 38.210884926434076}, {4482, 38.312564817548534}, {4483, 38.6074646961431}, {4484, 38.61409094278309}, {4485, 38.017765676373614}, {4486, 38.13255323384495}, {4487, 38.08371503820191}, {4488, 39.74784443380604}, {4489, 39.581076576624355}, + {4490, 39.560969587140264}, {4491, 39.633230280700445}, {4492, 39.4716793618942}, {4493, 39.85116292164824}, {4494, 40.13334579775557}, {4495, 40.63456793316489}, {4496, 40.71073257614338}, {4497, 40.614825837238605}, {4498, 40.52093195142476}, {4499, 40.625992282699016}, + {4500, 40.61279333802842}, {4501, 40.60312836705065}, {4502, 40.45124308705166}, {4503, 39.813179582460556}, {4504, 39.40056608371914}, {4505, 39.400036014144824}, {4506, 40.068292685250626}, {4507, 39.85851159258621}, {4508, 40.22426761068521}, {4509, 40.20179430857556}, + {4510, 40.195867059687664}, {4511, 40.17841644766825}, {4512, 40.33883934166413}, {4513, 40.43097043722167}, {4514, 39.97935686013634}, {4515, 40.35201602305827}, {4516, 40.362381301668016}, {4517, 41.646176394626345}, {4518, 41.090525592460445}, {4519, 40.79730768415044}, + {4520, 40.72921199361499}, {4521, 40.41889455210564}, {4522, 40.379424134550845}, {4523, 40.39436450202139}, {4524, 40.009568103338225}, {4525, 40.60519761105382}, {4526, 40.20213007918731}, {4527, 40.39015160569152}, {4528, 39.9454660982101}, {4529, 39.075001110176856}, + {4530, 39.13454982085495}, {4531, 39.648058502641085}, {4532, 39.648144119118776}, {4533, 39.324207361611386}, {4534, 39.53997526505957}, {4535, 39.97557044175399}, {4536, 40.26787350849}, {4537, 40.17369778686508}, {4538, 40.44006641312129}, {4539, 40.57083491561394}, + {4540, 40.66689326871952}, {4541, 40.65187955527909}, {4542, 39.934274637704824}, {4543, 40.300330136488476}, {4544, 39.967275063438045}, {4545, 40.042315039135865}, {4546, 40.142830573608606}, {4547, 40.061258001400915}, {4548, 40.404909259369646}, {4549, 40.450392865002364}, + {4550, 40.87269030449907}, {4551, 41.296599615123064}, {4552, 41.304588768416615}, {4553, 41.90167025496052}, {4554, 41.923025365981076}, {4555, 42.16170196977729}, {4556, 42.10361415553816}, {4557, 42.10129865988418}, {4558, 42.31197566235926}, {4559, 42.826247977993646}, + {4560, 42.74728359084352}, {4561, 42.667331833071664}, {4562, 42.928631961847074}, {4563, 43.00083178487252}, {4564, 43.37357490404957}, {4565, 43.2237675498371}, {4566, 43.86137004897719}, {4567, 43.98416288339857}, {4568, 44.0470084423368}, {4569, 44.11391700866567}, + {4570, 43.33351237600878}, {4571, 43.34757034996372}, {4572, 44.14608921239006}, {4573, 43.315144980691606}, {4574, 43.1647218607577}, {4575, 42.10202383999383}, {4576, 42.045575631273714}, {4577, 41.92172379039672}, {4578, 41.61112748037281}, {4579, 41.46655883593124}, + {4580, 42.135593115850845}, {4581, 42.28588201326873}, {4582, 42.15472962179464}, {4583, 41.752838975392386}, {4584, 41.91347248013341}, {4585, 41.92681361497359}, {4586, 41.9507666881852}, {4587, 41.90301229148557}, {4588, 41.95079936213436}, {4589, 42.01550498847925}, + {4590, 42.13012135355233}, {4591, 42.119576747364945}, {4592, 42.11677669401778}, {4593, 41.43172999572018}, {4594, 41.49388864519924}, {4595, 41.37380290015346}, {4596, 41.370035845679794}, {4597, 41.81317893158739}, {4598, 41.718756954520984}, {4599, 41.7298553763554}, + {4600, 41.53894095306525}, {4601, 41.75447883581544}, {4602, 41.885739012000045}, {4603, 42.06290639339678}, {4604, 42.01765532193647}, {4605, 42.09963693863422}, {4606, 42.121746882846026}, {4607, 42.40289053451183}, {4608, 42.43904810148321}, {4609, 42.073085450783054}, + {4610, 42.003884006662666}, {4611, 42.02391206710711}, {4612, 42.43143955801055}, {4613, 42.418444114505455}, {4614, 42.50130186535948}, {4615, 42.50861804985267}, {4616, 42.8667351445971}, {4617, 43.2708291680743}, {4618, 43.258875230182355}, {4619, 43.41983125615069}, + {4620, 43.02540169744918}, {4621, 43.03671781336059}, {4622, 43.51115813355301}, {4623, 43.620694664026104}, {4624, 44.28128377598449}, {4625, 45.28965233946238}, {4626, 45.984997240186246}, {4627, 45.98395822876192}, {4628, 46.06556368008116}, {4629, 45.196081738035325}, + {4630, 44.651596037353286}, {4631, 45.8360412664365}, {4632, 45.030975374089174}, {4633, 45.34523081375098}, {4634, 45.516897796124226}, {4635, 45.41290103045353}, {4636, 45.783086761335824}, {4637, 45.729375501632695}, {4638, 46.30543062423142}, {4639, 46.97672701088541}, + {4640, 46.47744540724059}, {4641, 46.43460283813155}, {4642, 46.176535515295534}, {4643, 46.1267773587671}, {4644, 46.07078753090914}, {4645, 45.90463206174146}, {4646, 45.55249821676164}, {4647, 46.10452178892155}, {4648, 46.657238306997954}, {4649, 46.676446405868745}, + {4650, 46.61931105547808}, {4651, 46.75145539959066}, {4652, 47.15614626698322}, {4653, 47.87910231233235}, {4654, 47.90315790426083}, {4655, 48.13748706690984}, {4656, 48.14012518669324}, {4657, 48.12478834534791}, {4658, 47.721710829839914}, {4659, 47.75882142791673}, + {4660, 48.62175299672685}, {4661, 49.44973560609625}, {4662, 49.45655590836775}, {4663, 49.467384079074314}, {4664, 49.36632082510971}, {4665, 49.361539089859946}, {4666, 48.923792508100405}, {4667, 48.82936960309324}, {4668, 48.696827533051106}, {4669, 49.021992536039356}, + {4670, 48.88792946529902}, {4671, 49.049698787091685}, {4672, 49.08423254107231}, {4673, 48.92829127058939}, {4674, 48.38289408354073}, {4675, 48.92579198716452}, {4676, 48.49132529951036}, {4677, 48.54115658361094}, {4678, 48.22070675260601}, {4679, 47.92838519785814}, + {4680, 47.732252711009075}, {4681, 45.99113534110142}, {4682, 44.72899549055614}, {4683, 44.55687451719093}, {4684, 44.55628471776198}, {4685, 44.44930616361875}, {4686, 44.69514501843518}, {4687, 44.08997798757562}, {4688, 44.326776424034165}, {4689, 44.5246373562509}, + {4690, 44.77024206609575}, {4691, 44.41279660589304}, {4692, 44.29090624408688}, {4693, 43.49452874948875}, {4694, 43.31906915186023}, {4695, 42.936317507114055}, {4696, 42.887727362033345}, {4697, 42.05490971051252}, {4698, 42.14072479814179}, {4699, 40.804558836186786}, + {4700, 41.48316460835661}, {4701, 41.52072176780903}, {4702, 41.5348100852975}, {4703, 41.75692669456157}, {4704, 41.49643711525131}, {4705, 42.35954965911466}, {4706, 42.254443486387316}, {4707, 42.346428230827634}, {4708, 42.26152211995296}, {4709, 42.978169988498536}, + {4710, 42.523208465380144}, {4711, 42.92136840228886}, {4712, 42.491877240874906}, {4713, 42.43808793916936}, {4714, 42.31386731961688}, {4715, 42.40503671098323}, {4716, 41.51956996493458}, {4717, 41.51666400077679}, {4718, 41.47426644424032}, {4719, 41.642970570173034}, + {4720, 41.62187072416654}, {4721, 41.15180912858265}, {4722, 41.894760156551904}, {4723, 42.092594714290364}, {4724, 42.27635546119988}, {4725, 42.69832068724076}, {4726, 42.980504524777594}, {4727, 45.87658310035213}, {4728, 43.6183451512828}, {4729, 43.92197152986545}, + {4730, 43.97403155795249}, {4731, 43.83776224600831}, {4732, 44.01998787494774}, {4733, 43.74137827823208}, {4734, 43.51842058845427}, {4735, 43.28245959362667}, {4736, 42.931398042043796}, {4737, 42.81785284737095}, {4738, 43.3507423739577}, {4739, 42.53626377548142}, + {4740, 43.32318432190919}, {4741, 44.1194336467924}, {4742, 44.256900789761175}, {4743, 44.471678262389915}, {4744, 44.52826323886761}, {4745, 44.28324388592612}, {4746, 44.23885158768107}, {4747, 44.118008732018836}, {4748, 44.126076154771745}, {4749, 43.920419005974225}, + {4750, 43.47296751071649}, {4751, 43.54310189974708}, {4752, 43.53066942512155}, {4753, 44.49097326057114}, {4754, 44.60107183314088}, {4755, 42.77090560264186}, {4756, 42.99334677650617}, {4757, 42.929199954491246}, {4758, 42.4663653797701}, {4759, 42.63510417353381}, + {4760, 41.15252150405892}, {4761, 41.65062666173321}, {4762, 41.63569595747878}, {4763, 41.80167602812174}, {4764, 41.54305663974481}, {4765, 41.09851635788159}, {4766, 40.44563966701578}, {4767, 40.533880450251715}, {4768, 40.8690019227734}, {4769, 40.88937124246862}, + {4770, 41.183809565163905}, {4771, 41.23702889676266}, {4772, 41.98448907911279}, {4773, 41.95032094100293}, {4774, 41.831147917269575}, {4775, 41.61384516565436}, {4776, 41.337855349523466}, {4777, 41.31691388756026}, {4778, 41.00824360651651}, {4779, 40.66303052923502}, + {4780, 40.75233960523218}, {4781, 40.74691723991323}, {4782, 41.224220277952135}, {4783, 41.24176898114373}, {4784, 41.19227291002064}, {4785, 41.455618057185966}, {4786, 41.793314227583664}, {4787, 41.169085632598424}, {4788, 41.147687532911064}, {4789, 41.308688926534245}, + {4790, 41.159786924004884}, {4791, 41.8142881123724}, {4792, 41.552546791536315}, {4793, 41.63707209178867}, {4794, 41.36580909924643}, {4795, 40.810470695252555}, {4796, 40.90861666010221}, {4797, 40.9449761604474}, {4798, 41.15181950532048}, {4799, 41.13789218005914}, + {4800, 41.08509141277718}, {4801, 41.05365891335565}, {4802, 40.980911652896054}, {4803, 41.734203454889105}, {4804, 41.31916007992194}, {4805, 41.4215682147432}, {4806, 41.51414417468774}, {4807, 42.37111482324612}, {4808, 42.70943804743709}, {4809, 42.666757607996296}, + {4810, 42.12147053040958}, {4811, 42.464100810348334}, {4812, 42.62139441696377}, {4813, 41.94637435030276}, {4814, 41.99684563449916}, {4815, 42.54816794861336}, {4816, 42.28223265087694}, {4817, 42.24550930900591}, {4818, 42.282491306075286}, {4819, 42.58423495563741}, + {4820, 41.359861111288104}, {4821, 42.10481353702761}, {4822, 42.24941093960088}, {4823, 41.19941990664588}, {4824, 41.21364184446058}, {4825, 41.72506184259513}, {4826, 42.53601771160175}, {4827, 42.67420054464686}, {4828, 42.76426057329836}, {4829, 42.78798548069922}, + {4830, 42.76148923124109}, {4831, 43.55500018064631}, {4832, 43.52818564137662}, {4833, 43.71030227628026}, {4834, 43.69559859751488}, {4835, 42.75390903216747}, {4836, 42.95452875778498}, {4837, 43.365066775687346}, {4838, 43.36814790382736}, {4839, 43.89333739845817}, + {4840, 43.989916759215}, {4841, 43.56253598246386}, {4842, 43.377743454747105}, {4843, 42.924400615882426}, {4844, 43.12304670119044}, {4845, 42.58288520924721}, {4846, 42.6192440085378}, {4847, 42.65060540788467}, {4848, 42.41222623035344}, {4849, 41.646360968682984}, + {4850, 41.424134917273825}, {4851, 41.10868413269653}, {4852, 41.172396017699}, {4853, 41.36004588804885}, {4854, 40.800077988280925}, {4855, 40.86912290176179}, {4856, 39.88125368753376}, {4857, 40.413562336579986}, {4858, 40.30739366800392}, {4859, 40.4448665834484}, + {4860, 40.67992036846289}, {4861, 39.19323041879348}, {4862, 38.934776988261994}, {4863, 38.6635282389286}, {4864, 38.018950718551935}, {4865, 38.04674518139207}, {4866, 38.01853292656019}, {4867, 37.71021607139222}, {4868, 38.422846767406455}, {4869, 38.450019969893354}, + {4870, 38.3997470792123}, {4871, 38.34361998177123}, {4872, 39.091980399463395}, {4873, 39.5198852558182}, {4874, 39.5472385581927}, {4875, 39.52236007766908}, {4876, 39.37969949452826}, {4877, 40.222868236832014}, {4878, 39.66661286220836}, {4879, 40.283042854444616}, + {4880, 40.229716416585205}, {4881, 40.445190873458}, {4882, 41.20372788803875}, {4883, 41.40919784184403}, {4884, 41.40205136110032}, {4885, 41.489596657482274}, {4886, 41.63320805924114}, {4887, 41.74501472339082}, {4888, 41.59463339227347}, {4889, 41.89165418164628}, + {4890, 42.429906756721266}, {4891, 42.51566334270849}, {4892, 42.43112651097444}, {4893, 42.65573412390896}, {4894, 42.5649774287204}, {4895, 42.20578627338075}, {4896, 42.19905186404266}, {4897, 42.063060621486514}, {4898, 40.8631606909062}, {4899, 41.4015851042295}, + {4900, 40.88944459737443}, {4901, 41.365721619037295}, {4902, 40.08190763242786}, {4903, 40.11091422145917}, {4904, 40.97087104137605}, {4905, 41.15805729486192}, {4906, 41.27896864220977}, {4907, 41.170130110377485}, {4908, 41.019095916064586}, {4909, 41.19823454109512}, + {4910, 40.71222352487721}, {4911, 40.604096585580145}, {4912, 40.22428726447587}, {4913, 38.86084836942058}, {4914, 38.70041746310922}, {4915, 38.8194506164692}, {4916, 38.64467182897344}, {4917, 38.639870243656844}, {4918, 38.929014450694986}, {4919, 38.80087908949201}, + {4920, 38.474735694735124}, {4921, 38.66300047873025}, {4922, 38.684455038039566}, {4923, 38.29265621342109}, {4924, 38.49017410077791}, {4925, 38.88748571310341}, {4926, 39.34561928924596}, {4927, 39.41078759839496}, {4928, 39.753432401888055}, {4929, 39.66725496587569}, + {4930, 39.64503039721447}, {4931, 39.74434633216348}, {4932, 39.24422485817575}, {4933, 39.357176347703124}, {4934, 39.083868743073175}, {4935, 38.77958154299146}, {4936, 38.784739437051876}, {4937, 38.84416930772779}, {4938, 38.531983904133995}, {4939, 38.13839982421345}, + {4940, 38.19834116299139}, {4941, 38.384878790359856}, {4942, 37.41788048991505}, {4943, 38.32825078053794}, {4944, 37.27571543485687}, {4945, 37.381643210593964}, {4946, 37.139400383368596}, {4947, 37.686425234248986}, {4948, 37.603432975092}, {4949, 37.64359097418195}, + {4950, 37.23329232367846}, {4951, 36.994849462220934}, {4952, 36.826705097979186}, {4953, 36.10850429296769}, {4954, 36.10515278531009}, {4955, 36.27585866868438}, {4956, 35.877106950223286}, {4957, 35.06902417472288}, {4958, 35.68954496717155}, {4959, 34.429975016070486}, + {4960, 33.9039081124664}, {4961, 34.50994734260132}, {4962, 34.24726508634311}, {4963, 34.27196566995411}, {4964, 34.53854603237626}, {4965, 33.18029169046981}, {4966, 33.74849740086445}, {4967, 33.7660814446232}, {4968, 33.690731394344716}, {4969, 33.75687045286519}, + {4970, 33.94382441333488}, {4971, 33.56341790269826}, {4972, 33.64387610469928}, {4973, 33.603199106999526}, {4974, 33.55490176741914}, {4975, 33.76379562158691}, {4976, 33.76315401597445}, {4977, 32.8267558087521}, {4978, 32.45241276131267}, {4979, 32.55733463964615}, + {4980, 33.319801723754786}, {4981, 33.59341126615778}, {4982, 33.73081923269426}, {4983, 34.37346764688176}, {4984, 35.07176548766099}, {4985, 35.37591959109057}, {4986, 35.471233571041964}, {4987, 35.13016122517179}, {4988, 34.65099719562258}, {4989, 34.37111074112498}, + {4990, 34.51523331821902}, {4991, 34.357189444832144}, {4992, 34.36407537164143}, {4993, 34.34124435265085}, {4994, 34.71701325993874}, {4995, 34.535111124444725}, {4996, 34.12819607955428}, {4997, 33.476805201968794}, {4998, 33.21364571883214}, {4999, 34.07622604782128}, +} + +// Point is a point on a line +type Point struct { + X float64 + Y float64 +} + +func BenchmarkLTTB(b *testing.B) { + ctx := context.New() + defer ctx.Close() + + seriesStart := time.Now() + stepSize := 1000 + values := NewValues(ctx, stepSize, len(data)) + for i := 0; i < values.Len(); i++ { + values.SetValueAt(i, data[i].Y) + } + series := NewSeries(ctx, "foo", seriesStart, values) + + for i := 0; i < b.N; i++ { + LTTB(series, series.StartTime(), series.EndTime(), 5000) + } +} + +func BenchmarkAverage(b *testing.B) { + ctx := context.New() + defer ctx.Close() + + seriesStart := time.Now() + stepSize := 1000 + values := NewValues(ctx, stepSize, len(data)) + for i := 0; i < values.Len(); i++ { + values.SetValueAt(i, data[i].Y) + } + series := NewSeries(ctx, "foo", seriesStart, values) + + for i := 0; i < b.N; i++ { + series.IntersectAndResize(series.StartTime(), series.EndTime(), 5000, Avg) + } +} diff --git a/src/query/graphite/ts/series.go b/src/query/graphite/ts/series.go new file mode 100644 index 0000000000..6db3defce3 --- /dev/null +++ b/src/query/graphite/ts/series.go @@ -0,0 +1,644 @@ +// Copyright (c) 2019 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 ts + +import ( + "errors" + "math" + "time" + + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/stats" +) + +var ( + // ErrRangeIsInvalid is returned when attempting to slice Series with invalid range + // endpoints (begin is beyond end). + ErrRangeIsInvalid = errors.New("requested range is invalid") +) + +// An AggregationFunc combines two data values at a given point. +type AggregationFunc func(a, b float64) float64 + +// A Series is the public interface to a block of timeseries values. Each block has a start time, +// a logical number of steps, and a step size indicating the number of milliseconds represented by each point. +type Series struct { + name string + startTime time.Time + vals Values + ctx context.Context + + // The Specification is the path that was used to generate this timeseries, + // typically either the query, or the function stack used to transform + // specific results. + Specification string + + // consolidationFunc specifies how the series will be consolidated when the + // number of data points in the series is more than the maximum number allowed. + consolidationFunc ConsolidationFunc +} + +// SeriesByName implements sort.Interface for sorting collections of series by name +type SeriesByName []*Series + +// Len returns the length of the series collection +func (a SeriesByName) Len() int { return len(a) } + +// Swap swaps two series in the collection +func (a SeriesByName) Swap(i, j int) { a[i], a[j] = a[j], a[i] } + +// Less determines if a series is ordered before another series by name +func (a SeriesByName) Less(i, j int) bool { return a[i].name < a[j].name } + +// NewSeries creates a new Series at a given start time, backed by the provided values +func NewSeries(ctx context.Context, name string, startTime time.Time, vals Values) *Series { + return &Series{ + name: name, + startTime: startTime, + vals: vals, + ctx: ctx, + Specification: name, + } +} + +// DerivedSeries returns a series derived from the current series with different datapoints +func (b *Series) DerivedSeries(startTime time.Time, vals Values) *Series { + series := NewSeries(b.ctx, b.name, startTime, vals) + series.Specification = b.Specification + series.consolidationFunc = b.consolidationFunc + return series +} + +// Name returns the name of the timeseries block +func (b *Series) Name() string { return b.name } + +// RenamedTo returns a new timeseries with the same values but a different name +func (b *Series) RenamedTo(name string) *Series { + return &Series{ + name: name, + startTime: b.startTime, + vals: b.vals, + ctx: b.ctx, + Specification: b.Specification, + consolidationFunc: b.consolidationFunc, + } +} + +// Shift returns a new timeseries with the same values but a different startTime +func (b *Series) Shift(shift time.Duration) *Series { + return &Series{ + name: b.name, + startTime: b.startTime.Add(shift), + vals: b.vals, + ctx: b.ctx, + Specification: b.Specification, + consolidationFunc: b.consolidationFunc, + } +} + +// StartTime returns the time the block starts +func (b *Series) StartTime() time.Time { return b.startTime } + +// EndTime returns the time the block ends +func (b *Series) EndTime() time.Time { return b.startTime.Add(b.Duration()) } + +// Duration returns the Duration covered by the block +func (b *Series) Duration() time.Duration { + return time.Millisecond * time.Duration(b.vals.Len()*b.vals.MillisPerStep()) +} + +// MillisPerStep returns the number of milliseconds per step +func (b *Series) MillisPerStep() int { return b.vals.MillisPerStep() } + +// Resolution returns resolution per step +func (b *Series) Resolution() time.Duration { + return time.Duration(b.MillisPerStep()) * time.Millisecond +} + +// StepAtTime returns the step within the block containing the given time +func (b *Series) StepAtTime(t time.Time) int { + return int(t.UnixNano()/1000000-b.startTime.UnixNano()/1000000) / b.vals.MillisPerStep() +} + +// StartTimeForStep returns the time at which the given step starts +func (b *Series) StartTimeForStep(n int) time.Time { + return b.StartTime().Add(time.Millisecond * time.Duration(n*b.vals.MillisPerStep())) +} + +// EndTimeForStep returns the time at which the given step end +func (b *Series) EndTimeForStep(n int) time.Time { + return b.StartTimeForStep(n).Add(time.Millisecond * time.Duration(b.vals.MillisPerStep())) +} + +// Slice returns a new Series composed from a subset of values in the original Series +func (b *Series) Slice(begin, end int) (*Series, error) { + if begin >= end { + return nil, ErrRangeIsInvalid + } + + result := NewSeries(b.ctx, b.name, b.StartTimeForStep(begin), b.vals.Slice(begin, end)) + result.consolidationFunc = b.consolidationFunc + + return result, nil +} + +// ValueAtTime returns the value stored at the step representing the given time +func (b *Series) ValueAtTime(t time.Time) float64 { + return b.ValueAt(b.StepAtTime(t)) +} + +// AllNaN returns true if the timeseries is all NaNs +func (b *Series) AllNaN() bool { return b.vals.AllNaN() } + +// CalcStatistics calculates a standard aggregation across the block values +func (b *Series) CalcStatistics() stats.Statistics { + if agg, ok := b.vals.(CustomStatistics); ok { + return agg.CalcStatistics() + } + + return stats.Calc(b) +} + +// Contains checks whether the given series contains the provided time +func (b *Series) Contains(t time.Time) bool { + step := b.StepAtTime(t) + return step >= 0 && step < b.Len() +} + +// Len returns the number of values in the time series. Used for aggregation +func (b *Series) Len() int { return b.vals.Len() } + +// ValueAt returns the value at a given step. Used for aggregation +func (b *Series) ValueAt(i int) float64 { return b.vals.ValueAt(i) } + +// SafeMax returns the maximum value of a series that's not an NaN. +func (b *Series) SafeMax() float64 { return b.CalcStatistics().Max } + +// SafeMin returns the minimum value of a series that's not an NaN. +func (b *Series) SafeMin() float64 { return b.CalcStatistics().Min } + +// SafeSum returns the sum of the values of a series, excluding NaNs. +func (b *Series) SafeSum() float64 { return b.CalcStatistics().Sum } + +// SafeAvg returns the average of the values of a series, excluding NaNs. +func (b *Series) SafeAvg() float64 { return b.CalcStatistics().Mean } + +// SafeStdDev returns the standard deviation of the values of a series, excluding NaNs. +func (b *Series) SafeStdDev() float64 { return b.CalcStatistics().StdDev } + +// SafeLastValue returns the last datapoint of a series that's not an NaN. +func (b *Series) SafeLastValue() float64 { + numPoints := b.Len() + for i := numPoints - 1; i >= 0; i-- { + v := b.ValueAt(i) + if !math.IsNaN(v) { + return v + } + } + return math.NaN() +} + +// SafeValues returns all non-NaN values in the series. +func (b *Series) SafeValues() []float64 { + numPoints := b.Len() + vals := make([]float64, 0, numPoints) + for i := 0; i < numPoints; i++ { + v := b.ValueAt(i) + if !math.IsNaN(v) { + vals = append(vals, v) + } + } + return vals +} + +// ConsolidationFunc returns the consolidation function for the series, +// or the averaging function is none specified. +func (b *Series) ConsolidationFunc() ConsolidationFunc { + if b.consolidationFunc != nil { + return b.consolidationFunc + } + return Avg +} + +// IsConsolidationFuncSet if the consolidationFunc is set +func (b *Series) IsConsolidationFuncSet() bool { + return b.consolidationFunc != nil +} + +// SetConsolidationFunc sets the consolidation function for the series +func (b *Series) SetConsolidationFunc(cf ConsolidationFunc) { + b.consolidationFunc = cf +} + +// PostConsolidationFunc is a function that takes a tuple of time and value after consolidation. +type PostConsolidationFunc func(timestamp time.Time, value float64) + +// intersection returns a 3-tuple; First return parameter indicates if the intersection spans at +// least one nanosecond; the next two return parameters are the start and end boundary timestamps +// of the resulting overlap. +func (b *Series) intersection(start, end time.Time) (bool, time.Time, time.Time) { + if b.EndTime().Before(start) || b.StartTime().After(end) { + return false, start, end + } + if start.Before(b.StartTime()) { + start = b.StartTime() + } + if end.After(b.EndTime()) { + end = b.EndTime() + } + if start.Equal(end) { + return false, start, end + } + return true, start, end +} + +// resize takes a time series and returns a new time series of a different step size with aggregated +// values; callers must provide callback method that collects the aggregated result +func (b *Series) resizeStep(start, end time.Time, millisPerStep int, + stepAggregator ConsolidationFunc, callback PostConsolidationFunc) { + // panic, panic, panic for all malformed callers + if end.Before(start) || start.Before(b.StartTime()) || end.After(b.EndTime()) { + panic("invalid boundary params") + } + if b.MillisPerStep() == millisPerStep { + panic("requires different step size") + } + if b.MillisPerStep() < millisPerStep { + // Series step size is smaller than consolidation - aggregate each series step then apply + // the agggregated value to the consolidate. + seriesValuesPerStep := millisPerStep / b.MillisPerStep() + seriesStart, seriesEnd := b.StepAtTime(start), b.StepAtTime(end) + for n := seriesStart; n < seriesEnd; n += seriesValuesPerStep { + timestamp := b.StartTimeForStep(n) + aggregatedValue := math.NaN() + count := 0 + + for i := 0; i < seriesValuesPerStep && n+i < seriesEnd; i++ { + value := b.ValueAt(n + i) + aggregatedValue, count = consolidateValues(aggregatedValue, value, count, + stepAggregator) + } + callback(timestamp, aggregatedValue) + } + return + } +} + +// resized implements PostConsolidationFunc. +type resized struct { + values []float64 +} + +// appender adds new values to resized.values. +func (v *resized) appender(timestamp time.Time, value float64) { + v.values = append(v.values, value) +} + +// IntersectAndResize returns a new time series with a different millisPerStep that spans the +// intersection of the underlying timeseries and the provided start and end time parameters +func (b *Series) IntersectAndResize(start, end time.Time, millisPerStep int, + stepAggregator ConsolidationFunc) (*Series, error) { + intersects, start, end := b.intersection(start, end) + if !intersects { + ts := NewSeries(b.ctx, b.name, start, &float64Values{ + millisPerStep: millisPerStep, + values: []float64{}, + numSteps: 0, + }) + ts.Specification = b.Specification + return ts, nil + } + if b.MillisPerStep() == millisPerStep { + return b.Slice(b.StepAtTime(start), b.StepAtTime(end)) + } + + // TODO: This append based model completely screws pooling; need to rewrite to allow for pooling. + v := &resized{} + b.resizeStep(start, end, millisPerStep, stepAggregator, v.appender) + ts := NewSeries(b.ctx, b.name, start, &float64Values{ + millisPerStep: millisPerStep, + values: v.values, + numSteps: len(v.values), + }) + ts.Specification = b.Specification + return ts, nil +} + +// A MutableSeries is a Series that allows updates +type MutableSeries struct { + Series +} + +// NewMutableSeries returns a new mutable Series at the +// given start time and backed by the provided storage +func NewMutableSeries( + ctx context.Context, + name string, + startTime time.Time, + vals MutableValues) *MutableSeries { + return &MutableSeries{ + Series{ + name: name, + startTime: startTime, + vals: vals, + ctx: ctx, + Specification: name, + }, + } +} + +// SetValueAt sets the value at the given step +func (b *MutableSeries) SetValueAt(i int, v float64) { + b.vals.(MutableValues).SetValueAt(i, v) +} + +// SetValueAtTime sets the value at the step containing the given time +func (b *MutableSeries) SetValueAtTime(t time.Time, v float64) { + b.SetValueAt(b.StepAtTime(t), v) +} + +// A Consolidation produces a Series whose values are the result of applying a consolidation +// function to all of the datapoints that fall within each step. It can used to quantize raw +// datapoints into a given resolution, for example, or to aggregate multiple timeseries at the +// same or smaller resolutions. +type Consolidation interface { + // AddDatapoint adds an individual datapoint to the consolidation. + AddDatapoint(timestamp time.Time, value float64) + + // AddDatapoints adds a set of datapoints to the consolidation. + AddDatapoints(datapoints []Datapoint) + + // AddSeries adds the datapoints for each series to the consolidation. The + // stepAggregationFunc is used to combine values from the series if the series + // has a smaller step size than the consolidation. For example, an application + // might want to produce a consolidation which is a minimum of the input timeseries, + // but where the values in smaller timeseries units are summed together to + // produce the value to which the consolidation applies. + // To put it in another way, stepAggregationFunc is used for the series to resize itself + // rather than for the consolidation + AddSeries(series *Series, stepAggregationFunc ConsolidationFunc) + + // BuildSeries returns the consolidated Series and optionally finalizes + // the consolidation returning it to the pool + BuildSeries(id string, finalize FinalizeOption) *Series + + // Finalize returns the consolidation to the pool + Finalize() +} + +// FinalizeOption specifies the option to finalize or avoid finalizing +type FinalizeOption int + +const ( + // NoFinalize will avoid finalizing the subject + NoFinalize FinalizeOption = iota + // Finalize will finalize the subject + Finalize +) + +// A ConsolidationFunc consolidates values at a given point in time. It takes the current consolidated +// value, the new value to add to the consolidation, and a count of the number of values that have +// already been consolidated. +type ConsolidationFunc func(existing, toAdd float64, count int) float64 + +// NewConsolidation creates a new consolidation window. +func NewConsolidation( + ctx context.Context, + start, end time.Time, + millisPerStep int, + cf ConsolidationFunc, +) Consolidation { + var ( + numSteps = NumSteps(start, end, millisPerStep) + values = NewValues(ctx, millisPerStep, numSteps) + c *consolidation + pooled = false + ) + + if consolidationPools != nil { + temp := consolidationPools.Get(numSteps) + c = temp.(*consolidation) + if cap(c.counts) >= numSteps { + c.counts = c.counts[:numSteps] + for i := range c.counts { + c.counts[i] = 0 + } + pooled = true + } + } + + if !pooled { + c = newConsolidation(numSteps) + } + + c.ctx = ctx + c.start = start + c.end = end + c.millisPerStep = millisPerStep + c.values = values + c.f = cf + + return c +} + +func newConsolidation(numSteps int) *consolidation { + counts := make([]int, numSteps) + return &consolidation{ + counts: counts, + } +} + +type consolidation struct { + ctx context.Context + start time.Time + end time.Time + millisPerStep int + values MutableValues + counts []int + f ConsolidationFunc +} + +func (c *consolidation) AddDatapoints(datapoints []Datapoint) { + for _, datapoint := range datapoints { + c.AddDatapoint(datapoint.Timestamp, datapoint.Value) + } +} + +func (c *consolidation) AddDatapoint(timestamp time.Time, value float64) { + if timestamp.Before(c.start) || timestamp.After(c.end) { + return + } + + if math.IsNaN(value) { + return + } + + step := int(timestamp.UnixNano()/1000000-c.start.UnixNano()/1000000) / c.millisPerStep + if step >= c.values.Len() { + return + } + + n, count := consolidateValues(c.values.ValueAt(step), value, c.counts[step], c.f) + c.counts[step] = count + c.values.SetValueAt(step, n) +} + +func consolidateValues(current, value float64, count int, f ConsolidationFunc) (float64, int) { + if math.IsNaN(value) { + return current, count + } + + if count == 0 { + return value, 1 + } + + return f(current, value, count), count + 1 +} + +// AddSeries adds a time series to the consolidation; stepAggregator is used to resize the +// provided timeseries if it's step size is different from the consolidator's step size. +func (c *consolidation) AddSeries(series *Series, stepAggregator ConsolidationFunc) { + if series.AllNaN() { + return + } + + intersects, start, end := series.intersection(c.start, c.end) + if !intersects { + // Nothing to do. + return + } + + if series.MillisPerStep() == c.millisPerStep { + // Series step size is identical to the consolidation: simply apply each series value to + // the consolidation. + startIndex := series.StepAtTime(start) + endIndex := int(math.Min(float64(series.StepAtTime(end)), float64(series.Len()-1))) + for n := startIndex; n <= endIndex; n++ { + c.AddDatapoint(series.StartTimeForStep(n), series.ValueAt(n)) + } + return + } + series.resizeStep(start, end, c.millisPerStep, stepAggregator, c.AddDatapoint) +} + +func (c *consolidation) BuildSeries(id string, f FinalizeOption) *Series { + series := NewSeries(c.ctx, id, c.start, c.values) + if f == Finalize { + c.Finalize() + } + return series +} + +func (c *consolidation) Finalize() { + c.ctx = nil + c.start = time.Time{} + c.end = time.Time{} + c.millisPerStep = 0 + c.values = nil + c.f = nil + if consolidationPools == nil { + return + } + consolidationPools.Put(c, cap(c.counts)) +} + +// NumSteps calculates the number of steps of a given size between two times. +func NumSteps(start, end time.Time, millisPerStep int) int { + // We should round up. + numSteps := int(math.Ceil(float64( + end.Sub(start)/time.Millisecond) / float64(millisPerStep))) + + if numSteps > 0 { + return numSteps + } + + // Even for intervals less than millisPerStep, there should be at least one step. + return 1 +} + +// Sum sums two values. +func Sum(a, b float64, count int) float64 { return a + b } + +// Mul multiplies two values. +func Mul(a, b float64, count int) float64 { return a * b } + +// Avg produces a running average. +func Avg(a, b float64, count int) float64 { return (a*float64(count) + b) / float64(count+1) } + +// Min finds the min of two values. +func Min(a, b float64, count int) float64 { return math.Min(a, b) } + +// Max finds the max of two values. +func Max(a, b float64, count int) float64 { return math.Max(a, b) } + +// Last finds the latter of two values. +func Last(a, b float64, count int) float64 { return b } + +// Gcd finds the gcd of two values. +func Gcd(a, b int64) int64 { + if a < 0 { + a = -a + } + + if b < 0 { + b = -b + } + + if b == 0 { + return a + } + + return Gcd(b, a%b) +} + +// Lcm finds the lcm of two values. +func Lcm(a, b int64) int64 { + if a < 0 { + a = -a + } + + if b < 0 { + b = -b + } + + if a == b { + return a + } + + if a < b { + a, b = b, a + } + + return a / Gcd(a, b) * b +} + +// A SeriesList is a list of series. +type SeriesList struct { + // Values is the list of series. + Values []*Series + // SortApplied specifies whether a specific sort order has been applied. + SortApplied bool +} + +// Len returns the length of the list. +func (l SeriesList) Len() int { + return len(l.Values) +} diff --git a/src/query/graphite/ts/series_reducer.go b/src/query/graphite/ts/series_reducer.go new file mode 100644 index 0000000000..6afa96a146 --- /dev/null +++ b/src/query/graphite/ts/series_reducer.go @@ -0,0 +1,64 @@ +// Copyright (c) 2019 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 ts + +import "fmt" + +// SeriesReducerApproach defines an approach to reduce a series to a single value. +type SeriesReducerApproach string + +// The standard set of reducers +const ( + SeriesReducerAvg SeriesReducerApproach = "avg" + SeriesReducerSum SeriesReducerApproach = "total" + SeriesReducerMin SeriesReducerApproach = "min" + SeriesReducerMax SeriesReducerApproach = "max" + SeriesReducerStdDev SeriesReducerApproach = "stddev" + SeriesReducerLast SeriesReducerApproach = "last" +) + +// SeriesReducer reduces a series to a single value. +type SeriesReducer func(*Series) float64 + +// SafeReducer returns a boolean indicating whether it is a valid reducer, +// and if so, the SeriesReducer implementing the SeriesReducerApproach. +func (sa SeriesReducerApproach) SafeReducer() (SeriesReducer, bool) { + r, ok := seriesReducers[sa] + return r, ok +} + +// Reducer returns the SeriesReducer implementing the SeriesReducerApproach. +func (sa SeriesReducerApproach) Reducer() SeriesReducer { + r, ok := sa.SafeReducer() + if !ok { + panic(fmt.Sprintf("No reducer func for %s", sa)) + } + return r +} + +var seriesReducers = map[SeriesReducerApproach]SeriesReducer{ + SeriesReducerAvg: func(b *Series) float64 { return b.SafeAvg() }, + SeriesReducerSum: func(b *Series) float64 { return b.SafeSum() }, + SeriesReducerMin: func(b *Series) float64 { return b.SafeMin() }, + SeriesReducerMax: func(b *Series) float64 { return b.SafeMax() }, + SeriesReducerStdDev: func(b *Series) float64 { return b.SafeStdDev() }, + SeriesReducerLast: func(b *Series) float64 { return b.SafeLastValue() }, +} diff --git a/src/query/graphite/ts/series_test.go b/src/query/graphite/ts/series_test.go new file mode 100644 index 0000000000..431f49d08b --- /dev/null +++ b/src/query/graphite/ts/series_test.go @@ -0,0 +1,480 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + "math/rand" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/context" + xtest "github.com/m3db/m3/src/query/graphite/testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +// A Datapoint is a datapoint (timestamp, value, optional series) used in testing +type testDatapoint struct { + SeriesName string + Timestamp time.Time + Value float64 +} + +// Datapoints is a set of datapoints +type testDatapoints []testDatapoint + +// Shuffle randomizes the set of datapoints +func (pts testDatapoints) Shuffle() { + for i := len(pts) - 1; i > 0; i-- { + if j := rand.Intn(i + 1); i != j { + pts[i], pts[j] = pts[j], pts[i] + } + } +} + +func TestLcm(t *testing.T) { + assert.Equal(t, int64(210), Lcm(10, 21)) + assert.Equal(t, int64(210), Lcm(10, -21)) + assert.Equal(t, int64(210), Lcm(-10, 21)) + assert.Equal(t, int64(210), Lcm(-10, -21)) + assert.Equal(t, int64(306), Lcm(18, 17)) + assert.Equal(t, int64(306), Lcm(17, 18)) + assert.Equal(t, int64(0), Lcm(0, 5)) +} + +func TestGcd(t *testing.T) { + assert.Equal(t, int64(5), Gcd(5, 10)) + assert.Equal(t, int64(5), Gcd(10, 5)) + assert.Equal(t, int64(5), Gcd(-10, 5)) + assert.Equal(t, int64(5), Gcd(10, -5)) + assert.Equal(t, int64(5), Gcd(-10, -5)) + assert.Equal(t, int64(10), Gcd(10, 10)) + assert.Equal(t, int64(8), Gcd(8, 0)) + assert.Equal(t, int64(8), Gcd(0, 8)) +} + +func TestAllNaN(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + values := NewValues(ctx, 1000, 30) + assert.True(t, values.AllNaN()) + values.SetValueAt(10, math.NaN()) + assert.True(t, values.AllNaN()) + values.SetValueAt(20, 100) + assert.False(t, values.AllNaN()) + + assert.True(t, NewConstantValues(ctx, math.NaN(), 1000, 10).AllNaN()) + assert.False(t, NewConstantValues(ctx, 200, 1000, 10).AllNaN()) +} + +func TestConstantValues(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + series := NewSeries(ctx, "foo", time.Now(), NewConstantValues(ctx, 100, 50, 1000)) + assert.Equal(t, 50, series.Len()) + n := series.ValueAt(10) + assert.Equal(t, float64(100), n) + + agg := series.CalcStatistics() + assert.Equal(t, uint(50), agg.Count) + assert.Equal(t, float64(100), agg.Min) + assert.Equal(t, float64(100), agg.Max) + assert.Equal(t, float64(100), agg.Mean) + assert.Equal(t, float64(0), agg.StdDev) +} + +func TestConstantNaNValues(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + series := NewSeries(ctx, "foo", time.Now(), NewConstantValues(ctx, math.NaN(), 50, 1000)) + assert.Equal(t, 50, series.Len()) + n := series.ValueAt(10) + assert.True(t, math.IsNaN(n)) + assert.False(t, series.IsConsolidationFuncSet()) + series.SetConsolidationFunc(ConsolidationSum.Func()) + assert.True(t, series.IsConsolidationFuncSet()) + xtest.Equalish(t, ConsolidationSum.Func(), series.consolidationFunc) + agg := series.CalcStatistics() + assert.Equal(t, uint(0), agg.Count) + assert.True(t, math.IsNaN(agg.Min)) + assert.True(t, math.IsNaN(agg.Max)) + assert.True(t, math.IsNaN(agg.Mean)) + assert.Equal(t, float64(0), agg.StdDev) +} + +func TestInvalidConsolidation(t *testing.T) { + var ( + ctx = context.New() + dummyCF = func(existing, toAdd float64, count int) float64 { + return existing + } + millisPerStep = 100 + start = time.Now() + end = start.Add(-1 * time.Hour) + ) + NewConsolidation(ctx, start, end, millisPerStep, dummyCF) +} + +func TestConsolidation(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + startTime := time.Now() + endTime := startTime.Add(5 * time.Minute) + + datapoints := testDatapoints{ + {Timestamp: startTime.Add(66 * time.Second), Value: 4.5}, + {Timestamp: startTime.Add(67 * time.Second), Value: 5.0}, + {Timestamp: startTime.Add(5 * time.Second), Value: 65.3}, + {Timestamp: startTime.Add(7 * time.Second), Value: 20.5}, + {Timestamp: startTime.Add(23 * time.Second), Value: 17.5}, + {Timestamp: startTime.Add(74 * time.Second), Value: 20.5}, + } + + consolidation := NewConsolidation(ctx, startTime, endTime, 10*1000, func(a, b float64, count int) float64 { + return a + b + }) + + for i := range datapoints { + consolidation.AddDatapoint(datapoints[i].Timestamp, datapoints[i].Value) + } + + series := consolidation.BuildSeries("foo", Finalize) + assert.Equal(t, 30, series.Len()) + statistics := series.CalcStatistics() + assert.Equal(t, float64(9.5), statistics.Min) // Sum of 66 and 67 second + assert.Equal(t, float64(85.8), statistics.Max) // Sum of 5 and 7 seconds + assert.Equal(t, uint(4), statistics.Count) // 66 and 67 are combined, 5 and 7 are combined + assert.Equal(t, float64(33.325), statistics.Mean) // Average of sums +} + +type consolidationTest struct { + name string + f ConsolidationFunc + stepAggregation ConsolidationFunc + expectedValues []float64 +} + +var ( + consolidationStartTime = time.Now().Truncate(time.Minute) + consolidationEndTime = consolidationStartTime.Add(1 * time.Minute) +) + +func newConsolidationTestSeries(ctx context.Context) []*Series { + return []*Series{ + // series1 starts and ends at the same time as the consolidation + NewSeries(ctx, "a", consolidationStartTime, + NewConstantValues(ctx, 10, 6, 10000)), + + // series2 starts before the consolidation but ends before the end + NewSeries(ctx, "b", consolidationStartTime.Add(-30*time.Second), + NewConstantValues(ctx, 15, 6, 10000)), + + // series3 starts after the consolidation and ends after the end + NewSeries(ctx, "c", consolidationStartTime.Add(30*time.Second), + NewConstantValues(ctx, 17, 6, 10000)), + + // series4 has a smaller step size than the consolidation + NewSeries(ctx, "d", consolidationStartTime, + NewConstantValues(ctx, 3, 60, 1000)), + } +} + +func TestConsolidateSeries(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + consolidatedSeries := newConsolidationTestSeries(ctx) + tests := []consolidationTest{ + {"sumMins", Sum, Min, []float64{28, 28, 28, 30, 30, 30}}, + {"minSums", Min, Sum, []float64{10, 10, 10, 10, 10, 10}}, + {"minMins", Min, Min, []float64{3, 3, 3, 3, 3, 3}}, + } + + for _, test := range tests { + consolidation := NewConsolidation(ctx, consolidationStartTime, consolidationEndTime, 10000, test.f) + for _, series := range consolidatedSeries { + consolidation.AddSeries(series, test.stepAggregation) + } + + results := consolidation.BuildSeries("foo", Finalize) + require.Equal(t, consolidationStartTime, results.StartTime(), "invalid start time for %s", test.name) + require.Equal(t, consolidationEndTime, results.EndTime(), "invalid end time for %s", test.name) + require.Equal(t, 6, results.Len(), "invalid consolidation size for %s", test.name) + + for i := 0; i < results.Len(); i++ { + value := results.ValueAt(i) + assert.Equal(t, test.expectedValues[i], value, "invalid value for %d of %s", i, test.name) + } + } +} + +func TestConsolidationAcrossTimeIntervals(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + expectedResults := []float64{ + 10 * 6, // entire range falls in the one minute period + 15 * 3, // last half falls into the one minute period + 17 * 3, // first half fallgs into the one minute period + 3 * 60, // entire range falls, at smaller interval + } + + consolidatedSeries := newConsolidationTestSeries(ctx) + for i := range consolidatedSeries { + series, expected := consolidatedSeries[i], expectedResults[i] + consolidation := NewConsolidation(ctx, consolidationStartTime, consolidationEndTime, 60000, Sum) + consolidation.AddSeries(series, Sum) + result := consolidation.BuildSeries("foo", Finalize) + assert.Equal(t, consolidationStartTime, result.StartTime(), "incorrect start to %s", series.Name()) + assert.Equal(t, consolidationEndTime, result.EndTime(), "incorrect end to %s", series.Name()) + require.Equal(t, 1, result.Len(), "incorrect # of steps for %s", series.Name()) + + value := result.ValueAt(0) + assert.Equal(t, expected, value, "incorrect value for %s", series.Name()) + + } +} + +func withValues(vals MutableValues, values []float64) Values { + for i, n := range values { + vals.SetValueAt(i, n) + } + return vals +} + +func TestSlicing(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + tests := []struct { + input Values + begin, end int + output Values + }{ + { + withValues(NewValues(ctx, 100, 5), []float64{10.0, 20.0, 30.0, 40.0, 50.0}), + 1, 3, + withValues(NewValues(ctx, 100, 2), []float64{20.0, 30.0}), + }, + { + NewConstantValues(ctx, 42.0, 10, 100), + 1, 5, + NewConstantValues(ctx, 42.0, 4, 100), + }, + } + + start := time.Now() + + for _, test := range tests { + input := NewSeries(ctx, "", start, test.input) + output, err := input.Slice(test.begin, test.end) + require.NoError(t, err) + + expect := NewSeries(ctx, "", input.StartTimeForStep(test.begin), test.output) + require.Equal(t, output.Len(), expect.Len()) + + for step := 0; step < output.Len(); step++ { + v1 := output.ValueAt(step) + v2 := expect.ValueAt(step) + assert.Equal(t, v1, v2) + } + } +} + +func TestAddSeries(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + seriesStart := time.Now().Add(10000 * time.Millisecond) + ctxStart := seriesStart.Add(400 * time.Millisecond) + ctxEnd := ctxStart.Add(7200 * time.Millisecond) + stepSize := 3600 + values := NewValues(ctx, stepSize, 3) + for i := 0; i < values.Len(); i++ { + values.SetValueAt(i, float64(i+1)) + } + series := NewSeries(ctx, "foo", seriesStart, values) + consolidation := NewConsolidation(ctx, ctxStart, ctxEnd, stepSize, Avg) + consolidation.AddSeries(series, Avg) + consolidated := consolidation.BuildSeries("consolidated", Finalize) + require.Equal(t, 2, consolidated.Len()) + require.Equal(t, 2.0, consolidated.ValueAt(0)) + require.Equal(t, 3.0, consolidated.ValueAt(1)) +} + +func TestIntersectAndResize(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + seriesStart := time.Now() + stepSize := 1000 + values := NewValues(ctx, stepSize, 3) + for i := 0; i < values.Len(); i++ { + values.SetValueAt(i, float64(i+1)) + } + series := NewSeries(ctx, "foo", seriesStart, values) + tests := []struct { + startOffset time.Duration + endOffset time.Duration + newStep int + }{ + { + startOffset: -1 * time.Hour, + endOffset: -1 * time.Hour, + newStep: stepSize, + }, + { + startOffset: 0, + endOffset: 0, + newStep: stepSize, + }, + { + startOffset: 0, + endOffset: 0, + newStep: stepSize / 2, + }, + } + + for _, test := range tests { + start := seriesStart.Add(test.startOffset) + end := seriesStart.Add(test.endOffset) + result, err := series.IntersectAndResize(start, end, test.newStep, series.ConsolidationFunc()) + require.NoError(t, err) + require.NotNil(t, result) + require.Equal(t, start, result.StartTime()) + require.Equal(t, end, result.EndTime()) + require.Equal(t, series.Specification, result.Specification) + require.Equal(t, series.Name(), result.Name()) + require.Equal(t, test.newStep, result.MillisPerStep()) + } +} + +var ( + benchmarkRange = 24 * time.Hour + benchmarkEndTime = time.Now() + benchmarkStartTime = benchmarkEndTime.Add(-benchmarkRange) + benchmarkNumSteps = NumSteps(benchmarkStartTime, benchmarkEndTime, benchmarkStepInMillis) +) + +const ( + benchmarkStepInMillis = 10000 +) + +func buildBenchmarkDatapoints() testDatapoints { + datapoints := make(testDatapoints, benchmarkNumSteps) + for i := range datapoints { + datapoints[i].Timestamp = benchmarkStartTime.Add(time.Millisecond * + time.Duration(i*benchmarkStepInMillis)) + datapoints[i].Value = 5 + } + + datapoints.Shuffle() + return datapoints +} + +func BenchmarkUint64Adds(b *testing.B) { + nan := math.Float64bits(math.NaN()) + datapoints := buildBenchmarkDatapoints() + for i := 0; i < b.N; i++ { + values := make([]uint64, len(datapoints)) + for j := 0; j < len(datapoints); j++ { + values[j] = nan + } + + for j := 0; j < len(datapoints); j++ { + startTimeMillis := benchmarkStartTime.UnixNano() / 1000000 + millis := datapoints[j].Timestamp.UnixNano() / 1000000 + + step := int(millis-startTimeMillis) / benchmarkStepInMillis + values[step] = 100 + 2 + } + } +} + +func BenchmarkFloat64Adds(b *testing.B) { + nan := math.NaN() + datapoints := buildBenchmarkDatapoints() + for i := 0; i < b.N; i++ { + values := make([]float64, len(datapoints)) + for j := 0; j < len(datapoints); j++ { + values[j] = nan + } + + for j := 0; j < len(datapoints); j++ { + startTimeMillis := benchmarkStartTime.UnixNano() / 1000000 + millis := datapoints[j].Timestamp.UnixNano() / 1000000 + + step := int(millis-startTimeMillis) / benchmarkStepInMillis + if math.IsNaN(values[step]) { + values[step] = 200 + } else { + values[step] = Sum(values[step], 200, 1) + } + } + } +} + +func BenchmarkConsolidation(b *testing.B) { + ctx := context.New() + defer ctx.Close() + + datapoints := buildBenchmarkDatapoints() + for i := 0; i < b.N; i++ { + consolidation := NewConsolidation(ctx, benchmarkStartTime, benchmarkEndTime, benchmarkStepInMillis, Sum) + for j := 0; j < len(datapoints); j++ { + consolidation.AddDatapoint(datapoints[j].Timestamp, datapoints[j].Value) + } + consolidation.BuildSeries("foo", Finalize) + } +} + +func BenchmarkConsolidationAddSeries(b *testing.B) { + ctx := context.New() + defer ctx.Close() + + c := NewConsolidation(ctx, benchmarkStartTime, benchmarkEndTime, benchmarkStepInMillis, Avg) + stepsInMillis := benchmarkStepInMillis / 10 + numSteps := int(benchmarkRange/time.Millisecond) / stepsInMillis + series := NewSeries(ctx, "a", benchmarkStartTime, + NewConstantValues(ctx, 3.1428, numSteps, stepsInMillis)) + + require.Equal(b, benchmarkStartTime, series.StartTime(), "start time not equal") + require.Equal(b, benchmarkEndTime, series.EndTime(), "end time not equal") + + for i := 0; i < b.N; i++ { + c.AddSeries(series, Sum) + } +} + +func BenchmarkNewSeries(b *testing.B) { + ctx := context.New() + defer ctx.Close() + for i := 0; i < b.N; i++ { + NewSeries(ctx, "a", benchmarkStartTime, + NewConstantValues(ctx, 3.1428, 1, 1000)) + } +} diff --git a/src/query/graphite/ts/sortable_series.go b/src/query/graphite/ts/sortable_series.go new file mode 100644 index 0000000000..fb5f39ab62 --- /dev/null +++ b/src/query/graphite/ts/sortable_series.go @@ -0,0 +1,81 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + "sort" +) + +// Direction signifies ascending or descending order +type Direction int + +const ( + // Ascending order + Ascending Direction = iota + // Descending order + Descending +) + +type sortableSeries struct { + series *Series + value float64 +} + +type sortableSeriesList []sortableSeries + +func (s sortableSeriesList) Len() int { return len(s) } + +func (s sortableSeriesList) Less(i, j int) bool { + if math.IsNaN(s[i].value) && !math.IsNaN(s[j].value) { + return true + } + return s[i].value < s[j].value +} + +func (s sortableSeriesList) Swap(i, j int) { + s[i], s[j] = s[j], s[i] +} + +// SortSeries applies a given SeriesReducer to each series in the input +// list and sorts based on the assigned value +func SortSeries(in []*Series, sr SeriesReducer, dir Direction) ([]*Series, error) { + var ( + sortableList = make(sortableSeriesList, 0, len(in)) + results = make([]*Series, len(in)) + ) + + for _, series := range in { + sortableList = append(sortableList, sortableSeries{series: series, value: sr(series)}) + } + + if dir == Ascending { + sort.Sort(sortableList) + } else { + sort.Sort(sort.Reverse(sortableList)) + } + + for i, sortable := range sortableList { + results[i] = sortable.series + } + + return results, nil +} diff --git a/src/query/graphite/ts/sortable_series_test.go b/src/query/graphite/ts/sortable_series_test.go new file mode 100644 index 0000000000..b146dad16c --- /dev/null +++ b/src/query/graphite/ts/sortable_series_test.go @@ -0,0 +1,133 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + "testing" + "time" + + "github.com/m3db/m3/src/query/graphite/context" + xtest "github.com/m3db/m3/src/query/graphite/testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +type testSeries struct { + name string + data []float64 +} + +type testSortData struct { + inputs []testSeries + output []testSeries +} + +func newTestSeriesValues(ctx context.Context, millisPerStep int, values []float64) Values { + tsv := NewValues(ctx, millisPerStep, len(values)) + + for i, n := range values { + tsv.SetValueAt(i, n) + } + + return tsv +} + +func newTestSeriesList(ctx context.Context, start time.Time, inputs []testSeries, step int) []*Series { + seriesList := make([]*Series, 0, len(inputs)) + + for _, in := range inputs { + series := NewSeries(ctx, in.name, start, newTestSeriesValues(ctx, step, in.data)) + seriesList = append(seriesList, series) + } + + return seriesList +} + +func validateOutputs(t *testing.T, step int, start time.Time, expected []testSeries, actual []*Series) { + require.Equal(t, len(expected), len(actual)) + + for i := range expected { + a, e := actual[i], expected[i].data + + require.Equal(t, len(e), a.Len()) + + for step := 0; step < a.Len(); step++ { + v := a.ValueAt(step) + xtest.Equalish(t, e[step], v, "invalid value for %d", step) + } + + assert.Equal(t, expected[i].name, a.Name()) + assert.Equal(t, step, a.MillisPerStep()) + assert.Equal(t, start, a.StartTime()) + } +} + +func testSortImpl(ctx context.Context, t *testing.T, tests []testSortData, sr SeriesReducer, dir Direction) { + var ( + startTime = time.Now() + step = 100 + ) + + for _, test := range tests { + series := newTestSeriesList(ctx, startTime, test.inputs, step) + + output, err := SortSeries(series, sr, dir) + + require.NoError(t, err) + validateOutputs(t, step, startTime, test.output, output) + } +} + +func TestSortSeries(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + testInput := []testSeries{ + {"foo", []float64{0, 601, 3, 4}}, + {"nan", []float64{math.NaN(), math.NaN(), math.NaN()}}, + {"bar", []float64{500, -8}}, + {"baz", []float64{600, -600, 3}}, + {"qux", []float64{100, 50000, 888, -1, -2}}, + } + + testSortImpl(ctx, t, []testSortData{ + {testInput, []testSeries{testInput[4], testInput[2], testInput[0], testInput[3], testInput[1]}}, + }, SeriesReducerAvg.Reducer(), Descending) + + testSortImpl(ctx, t, []testSortData{ + {testInput, []testSeries{testInput[0], testInput[3], testInput[4], testInput[2], testInput[1]}}, + }, SeriesReducerLast.Reducer(), Descending) + + testSortImpl(ctx, t, []testSortData{ + {testInput, []testSeries{testInput[4], testInput[0], testInput[3], testInput[2], testInput[1]}}, + }, SeriesReducerMax.Reducer(), Descending) + + testSortImpl(ctx, t, []testSortData{ + {testInput, []testSeries{testInput[4], testInput[3], testInput[2], testInput[0], testInput[1]}}, + }, SeriesReducerStdDev.Reducer(), Descending) + + testSortImpl(ctx, t, []testSortData{ + {testInput, []testSeries{testInput[1], testInput[3], testInput[0], testInput[2], testInput[4]}}, + }, SeriesReducerAvg.Reducer(), Ascending) + +} diff --git a/src/query/graphite/ts/values.go b/src/query/graphite/ts/values.go new file mode 100644 index 0000000000..e7599878bf --- /dev/null +++ b/src/query/graphite/ts/values.go @@ -0,0 +1,249 @@ +// Copyright (c) 2019 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 ts + +import ( + "math" + + "github.com/m3db/m3/src/query/graphite/context" + "github.com/m3db/m3/src/query/graphite/stats" + xts "github.com/m3db/m3/src/query/ts" + xpool "github.com/m3db/m3x/pool" +) + +// Values holds the values for a timeseries. It provides a minimal interface +// for storing and retrieving values in the series, with Series providing a +// more convenient interface for applications to build on top of. Values +// objects are not specific to a given time, allowing them to be +// pre-allocated, pooled, and re-used across multiple Series. There are +// multiple implementations of Values so that we can optimize storage based on +// the density of the series. +type Values interface { + stats.Values + + // The number of millisseconds represented by each index + MillisPerStep() int + + // Slice of data values in a range + Slice(begin, end int) Values + + // AllNaN returns true if the values are all NaN + AllNaN() bool +} + +// MutableValues is the interface for values that can be updated +type MutableValues interface { + Values + + // Resets the values + Reset() + + // Sets the value at the given entry + SetValueAt(n int, v float64) +} + +// CustomStatistics are for values that do custom statistics calculations +type CustomStatistics interface { + CalcStatistics() stats.Statistics +} + +// NewConstantValues returns a block of timeseries values all of which have the +// same value +func NewConstantValues(ctx context.Context, value float64, numSteps, millisPerStep int) Values { + return constantValues{ + numSteps: numSteps, + millisPerStep: millisPerStep, + value: value, + } +} + +type constantValues struct { + numSteps int + millisPerStep int + value float64 +} + +func (values constantValues) AllNaN() bool { return math.IsNaN(values.value) } +func (values constantValues) MillisPerStep() int { return values.millisPerStep } +func (values constantValues) Len() int { return values.numSteps } +func (values constantValues) ValueAt(point int) float64 { return values.value } +func (values constantValues) Slice(begin, end int) Values { + return &constantValues{ + end - begin, + values.millisPerStep, + values.value, + } +} + +func (values constantValues) CalcStatistics() stats.Statistics { + if math.IsNaN(values.value) { + return stats.Statistics{ + Count: 0, + StdDev: 0, + Min: math.NaN(), + Max: math.NaN(), + Mean: math.NaN(), + } + } + + return stats.Statistics{ + Count: uint(values.numSteps), + Min: values.value, + Max: values.value, + Mean: values.value, + StdDev: 0, + } +} + +// NewZeroValues returns a MutableValues supporting the given number of values +// at the requested granularity. The values start off initialized at 0 +func NewZeroValues(ctx context.Context, millisPerStep, numSteps int) MutableValues { + return newValues(ctx, millisPerStep, numSteps, 0) +} + +// NewValues returns MutableValues supporting the given number of values at the +// requested granularity. The values start off as NaN +func NewValues(ctx context.Context, millisPerStep, numSteps int) MutableValues { + return newValues(ctx, millisPerStep, numSteps, math.NaN()) +} + +var ( + pooledValuesLength = []int{} + pooledConsolidationsLength = []int{} +) + +var ( + timeSeriesValuesPools xpool.BucketizedObjectPool + consolidationPools xpool.BucketizedObjectPool +) + +func newValues(ctx context.Context, millisPerStep, numSteps int, initialValue float64) MutableValues { + var values []float64 + var pooled bool + + if timeSeriesValuesPools != nil { + temp := timeSeriesValuesPools.Get(numSteps) + values = temp.([]float64) + if cap(values) >= numSteps { + values = values[:numSteps] + pooled = true + } + } + + if !pooled { + values = make([]float64, numSteps) + } + + // Faster way to initialize an array instead of a loop + xts.Memset(values, initialValue) + vals := &float64Values{ + ctx: ctx, + millisPerStep: millisPerStep, + numSteps: numSteps, + allNaN: math.IsNaN(initialValue), + values: values, + } + ctx.RegisterCloser(vals) + return vals +} + +type float64Values struct { + ctx context.Context + millisPerStep int + numSteps int + values []float64 + allNaN bool +} + +func (b *float64Values) Reset() { + for i := range b.values { + b.values[i] = math.NaN() + } + b.allNaN = true +} + +func (b *float64Values) Close() error { + if timeSeriesValuesPools != nil { + timeSeriesValuesPools.Put(b.values, cap(b.values)) + } + b.numSteps = 0 + b.values = nil + return nil +} + +func (b *float64Values) AllNaN() bool { return b.allNaN } +func (b *float64Values) MillisPerStep() int { return b.millisPerStep } +func (b *float64Values) Len() int { return b.numSteps } +func (b *float64Values) ValueAt(point int) float64 { return b.values[point] } +func (b *float64Values) SetValueAt(point int, v float64) { + b.allNaN = b.allNaN && math.IsNaN(v) + b.values[point] = v +} + +func (b *float64Values) Slice(begin, end int) Values { + return &float64Values{ + ctx: b.ctx, + millisPerStep: b.millisPerStep, + values: b.values[begin:end], + numSteps: end - begin, + allNaN: false, // NB(mmihic): Someone might modify the parent and we won't be able to tell + } +} + +// PoolBucket is a pool bucket +type PoolBucket struct { + Capacity int + Count int +} + +func initPools(valueBuckets, consolidationBuckets []xpool.Bucket) error { + pooledValuesLength = pooledValuesLength[:0] + pooledConsolidationsLength = pooledConsolidationsLength[:0] + + for _, b := range valueBuckets { + pooledValuesLength = append(pooledValuesLength, b.Capacity) + } + for _, b := range consolidationBuckets { + pooledConsolidationsLength = append(pooledConsolidationsLength, b.Capacity) + } + + poolOpts := xpool.NewObjectPoolOptions() + valuesOpts := poolOpts.SetInstrumentOptions( + poolOpts.InstrumentOptions()) + consolidationOpts := poolOpts.SetInstrumentOptions( + poolOpts.InstrumentOptions()) + timeSeriesValuesPools = xpool.NewBucketizedObjectPool(valueBuckets, valuesOpts) + timeSeriesValuesPools.Init(func(capacity int) interface{} { + return make([]float64, capacity) + }) + consolidationPools = xpool.NewBucketizedObjectPool(consolidationBuckets, consolidationOpts) + consolidationPools.Init(func(capacity int) interface{} { + return newConsolidation(capacity) + }) + return nil +} + +// EnablePooling enables pooling. +func EnablePooling( + valueBuckets, consolidationBuckets []xpool.Bucket, +) { + initPools(valueBuckets, consolidationBuckets) +} diff --git a/src/query/graphite/ts/values_test.go b/src/query/graphite/ts/values_test.go new file mode 100644 index 0000000000..c25e716ab7 --- /dev/null +++ b/src/query/graphite/ts/values_test.go @@ -0,0 +1,41 @@ +// Copyright (c) 2019 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 ts + +import ( + "testing" + + "github.com/m3db/m3/src/query/graphite/context" + + "github.com/stretchr/testify/require" +) + +func TestValuesClose(t *testing.T) { + ctx := context.New() + defer ctx.Close() + + vals := newValues(ctx, 1000, 10, 0).(*float64Values) + err := vals.Close() + + require.NoError(t, err) + require.Equal(t, 0, vals.numSteps) + require.Nil(t, vals.values) +} diff --git a/src/query/storage/m3/multi_fetch_result.go b/src/query/storage/m3/multi_fetch_result.go index 74d14ffb15..078375546e 100644 --- a/src/query/storage/m3/multi_fetch_result.go +++ b/src/query/storage/m3/multi_fetch_result.go @@ -83,6 +83,30 @@ func (r *multiResult) Close() error { return nil } +func (r *multiResult) FinalResultWithAttrs() ( + encoding.SeriesIterators, []storage.Attributes, error) { + iters, err := r.FinalResult() + if err != nil { + return nil, nil, err + } + + attrs := make([]storage.Attributes, iters.Len()) + // TODO: add testing around here. + if r.dedupeMap == nil { + for i := range attrs { + attrs[i] = r.seenFirstAttrs + } + } else { + i := 0 + for _, res := range r.dedupeMap { + attrs[i] = res.attrs + i++ + } + } + + return iters, attrs, nil +} + func (r *multiResult) FinalResult() (encoding.SeriesIterators, error) { r.Lock() defer r.Unlock() diff --git a/src/query/storage/m3/storage.go b/src/query/storage/m3/storage.go index c139bf2b10..a47e1ea6b7 100644 --- a/src/query/storage/m3/storage.go +++ b/src/query/storage/m3/storage.go @@ -42,7 +42,9 @@ import ( ) var ( - errNoNamespacesConfigured = goerrors.New("no namespaces configured") + errNoNamespacesConfigured = goerrors.New("no namespaces configured") + errMismatchedFetchedLength = goerrors.New("length of fetched attributes and" + + " series iterators does not match") ) type queryFanoutType uint @@ -87,18 +89,37 @@ func (s *m3storage) Fetch( query *storage.FetchQuery, options *storage.FetchOptions, ) (*storage.FetchResult, error) { - raw, cleanup, err := s.FetchCompressed(ctx, query, options) - defer cleanup() + accumulator, err := s.fetchCompressed(ctx, query, options) + if err != nil { + return nil, err + } + + iters, attrs, err := accumulator.FinalResultWithAttrs() + defer accumulator.Close() if err != nil { return nil, err } - return storage.SeriesIteratorsToFetchResult( - raw, + fetchResult, err := storage.SeriesIteratorsToFetchResult( + iters, s.readWorkerPool, false, s.opts.TagOptions(), ) + + if err != nil { + return nil, err + } + + if len(fetchResult.SeriesList) != len(attrs) { + return nil, errMismatchedFetchedLength + } + + for i := range fetchResult.SeriesList { + fetchResult.SeriesList[i].SetResolution(attrs[i].Resolution) + } + + return fetchResult, nil } func (s *m3storage) FetchBlocks( @@ -150,16 +171,36 @@ func (s *m3storage) FetchCompressed( query *storage.FetchQuery, options *storage.FetchOptions, ) (encoding.SeriesIterators, Cleanup, error) { + accumulator, err := s.fetchCompressed(ctx, query, options) + if err != nil { + return nil, noop, err + } + + iters, err := accumulator.FinalResult() + if err != nil { + accumulator.Close() + return nil, noop, err + } + + return iters, accumulator.Close, nil +} + +// fetches compressed series, returning a MultiFetchResult accumulator +func (s *m3storage) fetchCompressed( + ctx context.Context, + query *storage.FetchQuery, + options *storage.FetchOptions, +) (MultiFetchResult, error) { // Check if the query was interrupted. select { case <-ctx.Done(): - return nil, noop, ctx.Err() + return nil, ctx.Err() default: } m3query, err := storage.FetchQueryToM3Query(query) if err != nil { - return nil, noop, err + return nil, err } // NB(r): Since we don't use a single index we fan out to each @@ -168,7 +209,7 @@ func (s *m3storage) FetchCompressed( // This needs to be optimized, however this is a start. fanout, namespaces, err := s.resolveClusterNamespacesForQuery(query.Start, query.End) if err != nil { - return nil, noop, err + return nil, err } var ( @@ -176,12 +217,12 @@ func (s *m3storage) FetchCompressed( wg sync.WaitGroup ) if len(namespaces) == 0 { - return nil, noop, errNoNamespacesConfigured + return nil, errNoNamespacesConfigured } pools, err := namespaces[0].Session().IteratorPools() if err != nil { - return nil, noop, fmt.Errorf("unable to retrieve iterator pools: %v", err) + return nil, fmt.Errorf("unable to retrieve iterator pools: %v", err) } result := newMultiFetchResult(fanout, pools) @@ -205,17 +246,11 @@ func (s *m3storage) FetchCompressed( // Check if the query was interrupted. select { case <-ctx.Done(): - return nil, noop, ctx.Err() + return nil, ctx.Err() default: } - iters, err := result.FinalResult() - if err != nil { - result.Close() - return nil, noop, err - } - - return iters, result.Close, nil + return result, err } func (s *m3storage) FetchTags( diff --git a/src/query/storage/m3/types.go b/src/query/storage/m3/types.go index d89eef9052..8f31dadc6a 100644 --- a/src/query/storage/m3/types.go +++ b/src/query/storage/m3/types.go @@ -59,18 +59,23 @@ type Querier interface { } // MultiFetchResult is a deduping accumalator for series iterators -// that allows merging using a given strategy +// that allows merging using a given strategy. type MultiFetchResult interface { - // Add adds series iterators with corresponding attributes to the accumulator + // Add adds series iterators with corresponding attributes to the accumulator. Add( attrs genericstorage.Attributes, iterators encoding.SeriesIterators, err error, ) // FinalResult returns a series iterators object containing - // deduplciated series values + // deduplicated series values. FinalResult() (encoding.SeriesIterators, error) - // Close releases all resources held by this accumulator + // FinalResultWithAttrs returns a series iterators object containing + // deduplicated series values, attributes corresponding to these iterators, + // and any errors encountered. + FinalResultWithAttrs() ( + encoding.SeriesIterators, []genericstorage.Attributes, error) + // Close releases all resources held by this accumulator. Close() error } diff --git a/src/query/test/m3/storage.go b/src/query/test/m3/test_storage.go similarity index 100% rename from src/query/test/m3/storage.go rename to src/query/test/m3/test_storage.go diff --git a/src/query/ts/series.go b/src/query/ts/series.go index 1adfbb8d1d..c44972fc7d 100644 --- a/src/query/ts/series.go +++ b/src/query/ts/series.go @@ -21,18 +21,22 @@ package ts import ( + "time" + "github.com/m3db/m3/src/query/models" ) -// Series is the public interface to a block of timeseries values. Each block has a start time, -// a logical number of steps, and a step size indicating the number of milliseconds represented by each point. +// Series is the public interface to a block of timeseries values. +// Each block has a start time, a logical number of steps, and a step size +// indicating the number of milliseconds represented by each point. type Series struct { - name string - vals Values - Tags models.Tags + resolution time.Duration + name string + vals Values + Tags models.Tags } -// NewSeries creates a new Series at a given start time, backed by the provided values +// NewSeries creates a new Series at a given start time, backed by the provided values. func NewSeries(name string, vals Values, tags models.Tags) *Series { return &Series{ name: name, @@ -44,11 +48,22 @@ func NewSeries(name string, vals Values, tags models.Tags) *Series { // Name returns the name of the timeseries block func (s *Series) Name() string { return s.name } -// Len returns the number of values in the time series. Used for aggregation +// Len returns the number of values in the time series. Used for aggregation. func (s *Series) Len() int { return s.vals.Len() } -// Values returns the underlying values interface +// Values returns the underlying values interface. func (s *Series) Values() Values { return s.vals } -// SeriesList represents a slice of series pointers +// Resolution retrieves the resolution for this series. +func (s *Series) Resolution() time.Duration { + return s.resolution +} + +// SetResolution sets the resolution for this series. Only used for +// graphite series consolidation logic after the fetch step. +func (s *Series) SetResolution(resolution time.Duration) { + s.resolution = resolution +} + +// SeriesList represents a slice of series pointers. type SeriesList []*Series