From 51689d965dbe3936bbc89def1e7a095286cbfda3 Mon Sep 17 00:00:00 2001 From: Julius Volz Date: Mon, 3 Jun 2013 17:07:03 +0200 Subject: [PATCH] Add debug timers to instant and range queries. This adds timers around several query-relevant code blocks. For now, the query timer stats are only logged for queries initiated through the UI. In other cases (rule evaluations), the stats are simply thrown away. My hope is that this helps us understand where queries spend time, especially in cases where they sometimes hang for unusual amounts of time. --- rules/alerting.go | 3 +- rules/ast/ast.go | 16 +++-- rules/ast/persistence_adapter.go | 12 +++- rules/ast/printer.go | 12 +++- rules/ast/query_analyzer.go | 29 +++++++-- rules/recording.go | 3 +- rules/rules_test.go | 3 +- stats/query_stats.go | 81 +++++++++++++++++++++++++ stats/timer.go | 100 +++++++++++++++++++++++++++++++ storage/metric/tiered.go | 15 ++++- storage/metric/tiered_test.go | 3 +- web/api/query.go | 25 ++++++-- 12 files changed, 280 insertions(+), 22 deletions(-) create mode 100644 stats/query_stats.go create mode 100644 stats/timer.go diff --git a/rules/alerting.go b/rules/alerting.go index 001b57737..1ab89f69e 100644 --- a/rules/alerting.go +++ b/rules/alerting.go @@ -17,6 +17,7 @@ import ( "fmt" "github.com/prometheus/prometheus/model" "github.com/prometheus/prometheus/rules/ast" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "github.com/prometheus/prometheus/utility" "time" @@ -90,7 +91,7 @@ type AlertingRule struct { func (rule AlertingRule) Name() string { return rule.name } func (rule AlertingRule) EvalRaw(timestamp time.Time, storage *metric.TieredStorage) (ast.Vector, error) { - return ast.EvalVectorInstant(rule.vector, timestamp, storage) + return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup()) } func (rule AlertingRule) Eval(timestamp time.Time, storage *metric.TieredStorage) (ast.Vector, error) { diff --git a/rules/ast/ast.go b/rules/ast/ast.go index 4d98b0ffb..2a9c8ace1 100644 --- a/rules/ast/ast.go +++ b/rules/ast/ast.go @@ -17,6 +17,7 @@ import ( "errors" "fmt" "github.com/prometheus/prometheus/model" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "log" "math" @@ -255,8 +256,8 @@ func labelsToKey(labels model.Metric) string { return strings.Join(keyParts, ",") // TODO not safe when label value contains comma. } -func EvalVectorInstant(node VectorNode, timestamp time.Time, storage *metric.TieredStorage) (vector Vector, err error) { - viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage) +func EvalVectorInstant(node VectorNode, timestamp time.Time, storage *metric.TieredStorage, queryStats *stats.TimerGroup) (vector Vector, err error) { + viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats) if err != nil { return } @@ -264,16 +265,20 @@ func EvalVectorInstant(node VectorNode, timestamp time.Time, storage *metric.Tie return } -func EvalVectorRange(node VectorNode, start time.Time, end time.Time, interval time.Duration, storage *metric.TieredStorage) (Matrix, error) { +func EvalVectorRange(node VectorNode, start time.Time, end time.Time, interval time.Duration, storage *metric.TieredStorage, queryStats *stats.TimerGroup) (Matrix, error) { // Explicitly initialize to an empty matrix since a nil Matrix encodes to // null in JSON. matrix := Matrix{} - viewAdapter, err := viewAdapterForRangeQuery(node, start, end, interval, storage) + viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start() + viewAdapter, err := viewAdapterForRangeQuery(node, start, end, interval, storage, queryStats) + viewTimer.Stop() if err != nil { return nil, err } + // TODO implement watchdog timer for long-running queries. + evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() sampleSets := map[string]*model.SampleSet{} for t := start; t.Before(end); t = t.Add(interval) { vector := node.Eval(t, viewAdapter) @@ -293,10 +298,13 @@ func EvalVectorRange(node VectorNode, start time.Time, end time.Time, interval t } } } + evalTimer.Stop() + appendTimer := queryStats.GetTimer(stats.ResultAppendTime).Start() for _, sampleSet := range sampleSets { matrix = append(matrix, *sampleSet) } + appendTimer.Stop() return matrix, nil } diff --git a/rules/ast/persistence_adapter.go b/rules/ast/persistence_adapter.go index 5d912fb14..87547d661 100644 --- a/rules/ast/persistence_adapter.go +++ b/rules/ast/persistence_adapter.go @@ -16,6 +16,7 @@ package ast import ( "flag" "github.com/prometheus/prometheus/model" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "time" ) @@ -39,6 +40,8 @@ type viewAdapter struct { // The materialized view which contains all timeseries data required for // executing a query. view metric.View + // The TimerGroup object in which to capture query timing statistics. + stats *stats.TimerGroup } // interpolateSamples interpolates a value at a target time between two @@ -105,6 +108,7 @@ func (v *viewAdapter) chooseClosestSample(samples model.Values, timestamp time.T } func (v *viewAdapter) GetValueAtTime(fingerprints model.Fingerprints, timestamp time.Time) (samples Vector, err error) { + timer := v.stats.GetTimer(stats.GetValueAtTimeTime).Start() for _, fingerprint := range fingerprints { sampleCandidates := v.view.GetValueAtTime(fingerprint, timestamp) samplePair := v.chooseClosestSample(sampleCandidates, timestamp) @@ -120,10 +124,12 @@ func (v *viewAdapter) GetValueAtTime(fingerprints model.Fingerprints, timestamp }) } } + timer.Stop() return samples, err } func (v *viewAdapter) GetBoundaryValues(fingerprints model.Fingerprints, interval *model.Interval) (sampleSets []model.SampleSet, err error) { + timer := v.stats.GetTimer(stats.GetBoundaryValuesTime).Start() for _, fingerprint := range fingerprints { samplePairs := v.view.GetBoundaryValues(fingerprint, *interval) if len(samplePairs) == 0 { @@ -142,10 +148,12 @@ func (v *viewAdapter) GetBoundaryValues(fingerprints model.Fingerprints, interva } sampleSets = append(sampleSets, sampleSet) } + timer.Stop() return sampleSets, nil } func (v *viewAdapter) GetRangeValues(fingerprints model.Fingerprints, interval *model.Interval) (sampleSets []model.SampleSet, err error) { + timer := v.stats.GetTimer(stats.GetRangeValuesTime).Start() for _, fingerprint := range fingerprints { samplePairs := v.view.GetRangeValues(fingerprint, *interval) if len(samplePairs) == 0 { @@ -164,10 +172,11 @@ func (v *viewAdapter) GetRangeValues(fingerprints model.Fingerprints, interval * } sampleSets = append(sampleSets, sampleSet) } + timer.Stop() return sampleSets, nil } -func NewViewAdapter(view metric.View, storage *metric.TieredStorage) *viewAdapter { +func NewViewAdapter(view metric.View, storage *metric.TieredStorage, queryStats *stats.TimerGroup) *viewAdapter { stalenessPolicy := StalenessPolicy{ DeltaAllowance: time.Duration(*defaultStalenessDelta) * time.Second, } @@ -176,5 +185,6 @@ func NewViewAdapter(view metric.View, storage *metric.TieredStorage) *viewAdapte stalenessPolicy: stalenessPolicy, storage: storage, view: view, + stats: queryStats, } } diff --git a/rules/ast/printer.go b/rules/ast/printer.go index 51bf81b67..a0c71672c 100644 --- a/rules/ast/printer.go +++ b/rules/ast/printer.go @@ -17,6 +17,7 @@ import ( "encoding/json" "fmt" "github.com/prometheus/prometheus/model" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "github.com/prometheus/prometheus/utility" "sort" @@ -153,15 +154,19 @@ func TypedValueToJSON(data interface{}, typeStr string) string { return string(dataJSON) } -func EvalToString(node Node, timestamp time.Time, format OutputFormat, storage *metric.TieredStorage) string { - viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage) +func EvalToString(node Node, timestamp time.Time, format OutputFormat, storage *metric.TieredStorage, queryStats *stats.TimerGroup) string { + viewTimer := queryStats.GetTimer(stats.TotalViewBuildingTime).Start() + viewAdapter, err := viewAdapterForInstantQuery(node, timestamp, storage, queryStats) + viewTimer.Stop() if err != nil { panic(err) } + evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() switch node.Type() { case SCALAR: scalar := node.(ScalarNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() switch format { case TEXT: return fmt.Sprintf("scalar: %v", scalar) @@ -170,6 +175,7 @@ func EvalToString(node Node, timestamp time.Time, format OutputFormat, storage * } case VECTOR: vector := node.(VectorNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() switch format { case TEXT: return vector.String() @@ -178,6 +184,7 @@ func EvalToString(node Node, timestamp time.Time, format OutputFormat, storage * } case MATRIX: matrix := node.(MatrixNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() switch format { case TEXT: return matrix.String() @@ -186,6 +193,7 @@ func EvalToString(node Node, timestamp time.Time, format OutputFormat, storage * } case STRING: str := node.(StringNode).Eval(timestamp, viewAdapter) + evalTimer.Stop() switch format { case TEXT: return str diff --git a/rules/ast/query_analyzer.go b/rules/ast/query_analyzer.go index 4afd96150..de39c0e5f 100644 --- a/rules/ast/query_analyzer.go +++ b/rules/ast/query_analyzer.go @@ -15,6 +15,7 @@ package ast import ( "github.com/prometheus/prometheus/model" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "log" "time" @@ -95,9 +96,13 @@ func (analyzer *QueryAnalyzer) AnalyzeQueries(node Node) { } } -func viewAdapterForInstantQuery(node Node, timestamp time.Time, storage *metric.TieredStorage) (*viewAdapter, error) { +func viewAdapterForInstantQuery(node Node, timestamp time.Time, storage *metric.TieredStorage, queryStats *stats.TimerGroup) (*viewAdapter, error) { + analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() analyzer := NewQueryAnalyzer(storage) analyzer.AnalyzeQueries(node) + analyzeTimer.Stop() + + requestBuildTimer := queryStats.GetTimer(stats.ViewRequestBuildTime).Start() viewBuilder := metric.NewViewRequestBuilder() for fingerprint, rangeDuration := range analyzer.FullRanges { viewBuilder.GetMetricRange(fingerprint, timestamp.Add(-rangeDuration), timestamp) @@ -105,16 +110,24 @@ func viewAdapterForInstantQuery(node Node, timestamp time.Time, storage *metric. for fingerprint := range analyzer.IntervalRanges { viewBuilder.GetMetricAtTime(fingerprint, timestamp) } - view, err := analyzer.storage.MakeView(viewBuilder, time.Duration(60)*time.Second) + requestBuildTimer.Stop() + + buildTimer := queryStats.GetTimer(stats.InnerViewBuildingTime).Start() + view, err := analyzer.storage.MakeView(viewBuilder, time.Duration(60)*time.Second, queryStats) + buildTimer.Stop() if err != nil { return nil, err } - return NewViewAdapter(view, storage), nil + return NewViewAdapter(view, storage, queryStats), nil } -func viewAdapterForRangeQuery(node Node, start time.Time, end time.Time, interval time.Duration, storage *metric.TieredStorage) (*viewAdapter, error) { +func viewAdapterForRangeQuery(node Node, start time.Time, end time.Time, interval time.Duration, storage *metric.TieredStorage, queryStats *stats.TimerGroup) (*viewAdapter, error) { + analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() analyzer := NewQueryAnalyzer(storage) analyzer.AnalyzeQueries(node) + analyzeTimer.Stop() + + requestBuildTimer := queryStats.GetTimer(stats.ViewRequestBuildTime).Start() viewBuilder := metric.NewViewRequestBuilder() for fingerprint, rangeDuration := range analyzer.FullRanges { // TODO: we should support GetMetricRangeAtInterval() or similar ops in the view builder. @@ -125,9 +138,13 @@ func viewAdapterForRangeQuery(node Node, start time.Time, end time.Time, interva for fingerprint := range analyzer.IntervalRanges { viewBuilder.GetMetricAtInterval(fingerprint, start, end, interval) } - view, err := analyzer.storage.MakeView(viewBuilder, time.Duration(60)*time.Second) + requestBuildTimer.Stop() + + buildTimer := queryStats.GetTimer(stats.InnerViewBuildingTime).Start() + view, err := analyzer.storage.MakeView(viewBuilder, time.Duration(60)*time.Second, queryStats) + buildTimer.Stop() if err != nil { return nil, err } - return NewViewAdapter(view, storage), nil + return NewViewAdapter(view, storage, queryStats), nil } diff --git a/rules/recording.go b/rules/recording.go index 6872b95be..7088c6d85 100644 --- a/rules/recording.go +++ b/rules/recording.go @@ -17,6 +17,7 @@ import ( "fmt" "github.com/prometheus/prometheus/model" "github.com/prometheus/prometheus/rules/ast" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "time" ) @@ -32,7 +33,7 @@ type RecordingRule struct { func (rule RecordingRule) Name() string { return rule.name } func (rule RecordingRule) EvalRaw(timestamp time.Time, storage *metric.TieredStorage) (ast.Vector, error) { - return ast.EvalVectorInstant(rule.vector, timestamp, storage) + return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup()) } func (rule RecordingRule) Eval(timestamp time.Time, storage *metric.TieredStorage) (ast.Vector, error) { diff --git a/rules/rules_test.go b/rules/rules_test.go index d2f9d8b23..e56d3bd6a 100644 --- a/rules/rules_test.go +++ b/rules/rules_test.go @@ -17,6 +17,7 @@ import ( "fmt" "github.com/prometheus/prometheus/model" "github.com/prometheus/prometheus/rules/ast" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/metric" "github.com/prometheus/prometheus/utility/test" "path" @@ -377,7 +378,7 @@ func TestExpressions(t *testing.T) { t.Errorf("%d. Test should fail, but didn't", i) } failed := false - resultStr := ast.EvalToString(testExpr, testEvalTime, ast.TEXT, tieredStorage) + resultStr := ast.EvalToString(testExpr, testEvalTime, ast.TEXT, tieredStorage, stats.NewTimerGroup()) resultLines := strings.Split(resultStr, "\n") if len(exprTest.output) != len(resultLines) { diff --git a/stats/query_stats.go b/stats/query_stats.go new file mode 100644 index 000000000..afbc406b0 --- /dev/null +++ b/stats/query_stats.go @@ -0,0 +1,81 @@ +// Copyright 2013 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stats + +// QueryTiming identifies the code area or functionality in which time is spent +// during a query. +type QueryTiming int + +// Query timings. +const ( + TotalEvalTime QueryTiming = iota + ResultSortTime + JsonEncodeTime + TotalViewBuildingTime + ViewRequestBuildTime + InnerViewBuildingTime + InnerEvalTime + ResultAppendTime + QueryAnalysisTime + GetValueAtTimeTime + GetBoundaryValuesTime + GetRangeValuesTime + ViewQueueTime + ViewDiskPreparationTime + ViewScanJobsTime + ViewDataExtractionTime + ViewDiskExtractionTime +) + +// Return a string represenation of a QueryTiming identifier. +func (s QueryTiming) String() string { + switch s { + case TotalEvalTime: + return "Total eval time" + case ResultSortTime: + return "Result sorting time" + case JsonEncodeTime: + return "JSON encoding time" + case TotalViewBuildingTime: + return "Total view building time" + case ViewRequestBuildTime: + return "View request building time" + case InnerViewBuildingTime: + return "Inner view building time" + case InnerEvalTime: + return "Inner eval time" + case ResultAppendTime: + return "Result append time" + case QueryAnalysisTime: + return "Query analysis time" + case GetValueAtTimeTime: + return "GetValueAtTime() time" + case GetBoundaryValuesTime: + return "GetBoundaryValues() time" + case GetRangeValuesTime: + return "GetRangeValues() time" + case ViewQueueTime: + return "View queue wait time" + case ViewScanJobsTime: + return "View building job scanning time" + case ViewDiskPreparationTime: + return "View building disk preparation time" + case ViewDataExtractionTime: + return "Total view data extraction time" + case ViewDiskExtractionTime: + return "View disk data extraction time" + default: + return "Unknown query timing" + } +} diff --git a/stats/timer.go b/stats/timer.go new file mode 100644 index 000000000..8d6ea8e88 --- /dev/null +++ b/stats/timer.go @@ -0,0 +1,100 @@ +// Copyright 2013 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stats + +import ( + "bytes" + "fmt" + "sort" + "time" +) + +// A timer that can be started and stopped and accumulates the total time it +// was running (the time between Start() and Stop()). +type Timer struct { + name fmt.Stringer + created time.Time + start time.Time + duration time.Duration +} + +// Start the timer. +func (t *Timer) Start() *Timer { + t.start = time.Now() + return t +} + +// Stop the timer. +func (t *Timer) Stop() { + t.duration += time.Since(t.start) +} + +// Return a string representation of the Timer. +func (t *Timer) String() string { + return fmt.Sprintf("%s: %s", t.name, t.duration) +} + +// A TimerGroup represents a group of timers relevant to a single query. +type TimerGroup struct { + timers map[fmt.Stringer]*Timer + child *TimerGroup +} + +// Construct a new TimerGroup. +func NewTimerGroup() *TimerGroup { + return &TimerGroup{timers: map[fmt.Stringer]*Timer{}} +} + +// Get (and create, if necessary) the Timer for a given code section. +func (t *TimerGroup) GetTimer(name fmt.Stringer) *Timer { + if timer, exists := t.timers[name]; exists { + return timer + } + timer := &Timer{ + name: name, + created: time.Now(), + } + t.timers[name] = timer + return timer +} + +type Timers []*Timer + +type byCreationTimeSorter struct{ Timers } + +func (t Timers) Len() int { + return len(t) +} + +func (t Timers) Swap(i, j int) { + t[i], t[j] = t[j], t[i] +} + +func (s byCreationTimeSorter) Less(i, j int) bool { + return s.Timers[i].created.Before(s.Timers[j].created) +} + +// Return a string representation of a TimerGroup. +func (t *TimerGroup) String() string { + timers := byCreationTimeSorter{} + for _, timer := range t.timers { + timers.Timers = append(timers.Timers, timer) + } + sort.Sort(timers) + result := &bytes.Buffer{} + for _, timer := range timers.Timers { + fmt.Fprintf(result, "%s\n", timer) + } + return result.String() +} diff --git a/storage/metric/tiered.go b/storage/metric/tiered.go index b84fba8d1..e37a5b105 100644 --- a/storage/metric/tiered.go +++ b/storage/metric/tiered.go @@ -25,6 +25,7 @@ import ( "github.com/prometheus/prometheus/coding" "github.com/prometheus/prometheus/coding/indexable" "github.com/prometheus/prometheus/model" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/storage/raw/leveldb" ) @@ -77,6 +78,7 @@ type viewJob struct { output chan View abort chan bool err chan error + stats *stats.TimerGroup } func NewTieredStorage(appendToDiskQueueDepth, viewQueueDepth uint, flushMemoryInterval, memoryTTL time.Duration, root string) (storage *TieredStorage, err error) { @@ -120,7 +122,7 @@ func (t *TieredStorage) Drain() { } // Enqueues a ViewRequestBuilder for materialization, subject to a timeout. -func (t *TieredStorage) MakeView(builder ViewRequestBuilder, deadline time.Duration) (View, error) { +func (t *TieredStorage) MakeView(builder ViewRequestBuilder, deadline time.Duration, queryStats *stats.TimerGroup) (View, error) { if len(t.draining) > 0 { return nil, fmt.Errorf("Storage is in the process of draining.") } @@ -133,11 +135,13 @@ func (t *TieredStorage) MakeView(builder ViewRequestBuilder, deadline time.Durat // has already exited and doesn't consume from the channel anymore. abortChan := make(chan bool, 1) errChan := make(chan error) + queryStats.GetTimer(stats.ViewQueueTime).Start() t.viewQueue <- viewJob{ builder: builder, output: result, abort: abortChan, err: errChan, + stats: queryStats, } select { @@ -169,6 +173,7 @@ func (t *TieredStorage) Serve() { case <-flushMemoryTicker.C: t.flushMemory(t.memoryTTL) case viewRequest := <-t.viewQueue: + viewRequest.stats.GetTimer(stats.ViewQueueTime).Stop() t.renderView(viewRequest) case drainingDone := <-t.draining: t.Flush() @@ -260,13 +265,16 @@ func (t *TieredStorage) renderView(viewJob viewJob) { recordOutcome(duration, err, map[string]string{operation: renderView, result: success}, map[string]string{operation: renderView, result: failure}) }() + scanJobsTimer := viewJob.stats.GetTimer(stats.ViewScanJobsTime).Start() scans := viewJob.builder.ScanJobs() + scanJobsTimer.Stop() view := newView() var iterator leveldb.Iterator = nil var diskFrontier *diskFrontier = nil var diskPresent = true + extractionTimer := viewJob.stats.GetTimer(stats.ViewDataExtractionTime).Start() for _, scanJob := range scans { var seriesFrontier *seriesFrontier = nil var seriesPresent = true @@ -286,6 +294,7 @@ func (t *TieredStorage) renderView(viewJob viewJob) { currentChunk := chunk{} // If we aimed before the oldest value in memory, load more data from disk. if (len(memValues) == 0 || memValues.FirstTimeAfter(targetTime)) && diskPresent && seriesPresent { + diskPrepareTimer := viewJob.stats.GetTimer(stats.ViewDiskPreparationTime).Start() // Conditionalize disk access. if diskFrontier == nil && diskPresent { if iterator == nil { @@ -310,9 +319,12 @@ func (t *TieredStorage) renderView(viewJob viewJob) { panic(err) } } + diskPrepareTimer.Stop() if diskPresent && seriesPresent { + diskTimer := viewJob.stats.GetTimer(stats.ViewDiskExtractionTime).Start() diskValues := t.loadChunkAroundTime(iterator, seriesFrontier, scanJob.fingerprint, targetTime) + diskTimer.Stop() // If we aimed past the newest value on disk, combine it with the next value from memory. if len(memValues) > 0 && diskValues.LastTimeBefore(targetTime) { @@ -381,6 +393,7 @@ func (t *TieredStorage) renderView(viewJob viewJob) { sort.Sort(startsAtSort{standingOps}) } } + extractionTimer.Stop() viewJob.output <- view return diff --git a/storage/metric/tiered_test.go b/storage/metric/tiered_test.go index 2cc02fb8e..785416ffd 100644 --- a/storage/metric/tiered_test.go +++ b/storage/metric/tiered_test.go @@ -15,6 +15,7 @@ package metric import ( "github.com/prometheus/prometheus/model" + "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/utility/test" "sort" "testing" @@ -363,7 +364,7 @@ func testMakeView(t test.Tester, flushToDisk bool) { requestBuilder.GetMetricRange(fingerprint, alongRange.from, alongRange.through) } - v, err := tiered.MakeView(requestBuilder, time.Second*5) + v, err := tiered.MakeView(requestBuilder, time.Second*5, stats.NewTimerGroup()) if err != nil { t.Fatalf("%d. failed due to %s", i, err) diff --git a/web/api/query.go b/web/api/query.go index 9c8f40fda..085dfef46 100644 --- a/web/api/query.go +++ b/web/api/query.go @@ -20,6 +20,7 @@ import ( "github.com/prometheus/prometheus/model" "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/rules/ast" + "github.com/prometheus/prometheus/stats" "log" "net/http" "sort" @@ -33,7 +34,7 @@ func (serv MetricsService) setAccessControlHeaders(rb *gorest.ResponseBuilder) { rb.AddHeader("Access-Control-Expose-Headers", "Date") } -func (serv MetricsService) Query(expr string, formatJson string) (result string) { +func (serv MetricsService) Query(expr string, formatJson string) string { exprNode, err := rules.LoadExprFromString(expr) if err != nil { return ast.ErrorToJSON(err) @@ -52,7 +53,10 @@ func (serv MetricsService) Query(expr string, formatJson string) (result string) rb.SetContentType(gorest.Text_Plain) } - return ast.EvalToString(exprNode, timestamp, format, serv.Storage) + queryStats := stats.NewTimerGroup() + result := ast.EvalToString(exprNode, timestamp, format, serv.Storage, queryStats) + log.Printf("Instant query: %s\nQuery stats:\n%s\n", expr, queryStats) + return result } func (serv MetricsService) QueryRange(expr string, end int64, duration int64, step int64) string { @@ -82,18 +86,31 @@ func (serv MetricsService) QueryRange(expr string, end int64, duration int64, st // Align the start to step "tick" boundary. end -= end % step + queryStats := stats.NewTimerGroup() + + evalTimer := queryStats.GetTimer(stats.TotalEvalTime).Start() matrix, err := ast.EvalVectorRange( exprNode.(ast.VectorNode), time.Unix(end-duration, 0).UTC(), time.Unix(end, 0).UTC(), time.Duration(step)*time.Second, - serv.Storage) + serv.Storage, + queryStats) if err != nil { return ast.ErrorToJSON(err) } + evalTimer.Stop() + sortTimer := queryStats.GetTimer(stats.ResultSortTime).Start() sort.Sort(matrix) - return ast.TypedValueToJSON(matrix, "matrix") + sortTimer.Stop() + + jsonTimer := queryStats.GetTimer(stats.JsonEncodeTime).Start() + result := ast.TypedValueToJSON(matrix, "matrix") + jsonTimer.Stop() + + log.Printf("Range query: %s\nQuery stats:\n%s\n", expr, queryStats) + return result } func (serv MetricsService) Metrics() string {