Skip to content

Commit

Permalink
Query: Add +Inf bucket to query duration metrics (#6358)
Browse files Browse the repository at this point in the history
* Query: Add +Inf bucket to query duration metrics

For the query duration metrics
(`thanos_store_api_query_duration_seconds`), we record query respond
latency, based on the size of the query (samples/series), and save to a
histogram.

However, when a query is made which exceeds the biggest sample/serie
size, we would prior to this commit, put the request into the largest
bucket.

With this commit, we instead create an `+Inf` bucket, and put requests
which are larger than the biggest defined bucket into that. This gives
more accurate results, and also allow one to see if the bucket sizes are
incorrectly sized.

Signed-off-by: Jacob Baungard Hansen <jacobbaungard@redhat.com>

* Tests: Mutex around non-thread safe random source

When creating test blocks, we use a non-thread safe random source, in
multiple goroutines. Due to this, tests would sometime panic.

This commits puts a mutex around calls using the same source, in order
to avoid this.

This should hopefully improve reliability of e2e tests.

Signed-off-by: Jacob Baungard Hansen <jacobbaungard@redhat.com>

---------

Signed-off-by: Jacob Baungard Hansen <jacobbaungard@redhat.com>
  • Loading branch information
jacobbaungard committed May 15, 2023
1 parent 0c3cda9 commit f6dfd7f
Show file tree
Hide file tree
Showing 6 changed files with 79 additions and 24 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ We use *breaking :warning:* to mark changes that are not backward compatible (re
- [#6163](https://github.com/thanos-io/thanos/pull/6163) Receiver: changed max backoff from 30s to 5s for forwarding requests. Can be configured with `--receive-forward-max-backoff`.
- [#6327](https://github.com/thanos-io/thanos/pull/6327) *: *breaking :warning:* Use histograms instead of summaries for instrumented handlers.
- [#6322](https://github.com/thanos-io/thanos/pull/6322) Logging: Avoid expensive log.Valuer evaluation for disallowed levels.
- [#6358](https://github.com/thanos-io/thanos/pull/6358) Query: Add +Inf bucket to query duration metrics

### Removed

Expand Down
8 changes: 4 additions & 4 deletions cmd/thanos/query.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,8 +215,8 @@ func registerQuery(app *extkingpin.App) {
grpcProxyStrategy := cmd.Flag("grpc.proxy-strategy", "Strategy to use when proxying Series requests to leaf nodes. Hidden and only used for testing, will be removed after lazy becomes the default.").Default(string(store.EagerRetrieval)).Hidden().Enum(string(store.EagerRetrieval), string(store.LazyRetrieval))

queryTelemetryDurationQuantiles := cmd.Flag("query.telemetry.request-duration-seconds-quantiles", "The quantiles for exporting metrics about the request duration quantiles.").Default("0.1", "0.25", "0.75", "1.25", "1.75", "2.5", "3", "5", "10").Float64List()
queryTelemetrySamplesQuantiles := cmd.Flag("query.telemetry.request-samples-quantiles", "The quantiles for exporting metrics about the samples count quantiles.").Default("100", "1000", "10000", "100000", "1000000").Int64List()
queryTelemetrySeriesQuantiles := cmd.Flag("query.telemetry.request-series-seconds-quantiles", "The quantiles for exporting metrics about the series count quantiles.").Default("10", "100", "1000", "10000", "100000").Int64List()
queryTelemetrySamplesQuantiles := cmd.Flag("query.telemetry.request-samples-quantiles", "The quantiles for exporting metrics about the samples count quantiles.").Default("100", "1000", "10000", "100000", "1000000").Float64List()
queryTelemetrySeriesQuantiles := cmd.Flag("query.telemetry.request-series-seconds-quantiles", "The quantiles for exporting metrics about the series count quantiles.").Default("10", "100", "1000", "10000", "100000").Float64List()

var storeRateLimits store.SeriesSelectLimits
storeRateLimits.RegisterFlags(cmd)
Expand Down Expand Up @@ -408,8 +408,8 @@ func runQuery(
grpcProxyStrategy string,
comp component.Component,
queryTelemetryDurationQuantiles []float64,
queryTelemetrySamplesQuantiles []int64,
queryTelemetrySeriesQuantiles []int64,
queryTelemetrySamplesQuantiles []float64,
queryTelemetrySeriesQuantiles []float64,
defaultEngine string,
storeRateLimits store.SeriesSelectLimits,
queryMode queryMode,
Expand Down
35 changes: 18 additions & 17 deletions pkg/store/telemetry.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@
package store

import (
"sort"
"strconv"

"github.com/prometheus/client_golang/prometheus"
Expand All @@ -17,17 +18,17 @@ import (
type seriesStatsAggregator struct {
queryDuration *prometheus.HistogramVec

seriesLeBuckets []int64
samplesLeBuckets []int64
seriesLeBuckets []float64
samplesLeBuckets []float64
seriesStats storepb.SeriesStatsCounter
}

// NewSeriesStatsAggregator is a constructor for seriesStatsAggregator.
func NewSeriesStatsAggregator(
reg prometheus.Registerer,
durationQuantiles []float64,
sampleQuantiles []int64,
seriesQuantiles []int64,
sampleQuantiles []float64,
seriesQuantiles []float64,
) *seriesStatsAggregator {
return &seriesStatsAggregator{
queryDuration: promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{
Expand All @@ -54,11 +55,11 @@ func (s *seriesStatsAggregator) Observe(duration float64) {
return
}
// Bucket matching for series/labels matchSeriesBucket/matchSamplesBucket => float64, float64
seriesLeBucket := s.findBucket(float64(s.seriesStats.Series), s.seriesLeBuckets)
samplesLeBucket := s.findBucket(float64(s.seriesStats.Samples), s.samplesLeBuckets)
seriesLeBucket := findBucket(float64(s.seriesStats.Series), s.seriesLeBuckets)
samplesLeBucket := findBucket(float64(s.seriesStats.Samples), s.samplesLeBuckets)
s.queryDuration.With(prometheus.Labels{
"series_le": strconv.Itoa(int(seriesLeBucket)),
"samples_le": strconv.Itoa(int(samplesLeBucket)),
"series_le": seriesLeBucket,
"samples_le": samplesLeBucket,
}).Observe(duration)
s.reset()
}
Expand All @@ -67,18 +68,18 @@ func (s *seriesStatsAggregator) reset() {
s.seriesStats = storepb.SeriesStatsCounter{}
}

func (s *seriesStatsAggregator) findBucket(value float64, quantiles []int64) int64 {
func findBucket(value float64, quantiles []float64) string {
if len(quantiles) == 0 {
return 0
return "+Inf"
}
var foundBucket int64
for _, bucket := range quantiles {
foundBucket = bucket
if value < float64(bucket) {
break
}

// If the value is bigger than the largest bucket we return +Inf
if value >= float64(quantiles[len(quantiles)-1]) {
return "+Inf"
}
return foundBucket

// SearchFloats64s gets the appropriate index in the quantiles array based on the value
return strconv.FormatFloat(quantiles[sort.SearchFloat64s(quantiles, value)], 'f', -1, 64)
}

// NoopSeriesStatsAggregator is a query performance series aggregator that does nothing.
Expand Down
4 changes: 4 additions & 0 deletions pkg/testutil/e2eutil/prometheus.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"runtime"
"sort"
"strings"
"sync"
"syscall"
"testing"
"time"
Expand Down Expand Up @@ -485,6 +486,7 @@ func createBlock(
var timeStepSize = (maxt - mint) / int64(numSamples+1)
var batchSize = len(series) / runtime.GOMAXPROCS(0)
r := rand.New(rand.NewSource(int64(numSamples)))
var randMutex sync.Mutex

for len(series) > 0 {
l := batchSize
Expand All @@ -507,7 +509,9 @@ func createBlock(

var err error
if sampleType == chunkenc.ValFloat {
randMutex.Lock()
_, err = app.Append(0, lset, t, r.Float64())
randMutex.Unlock()
} else if sampleType == chunkenc.ValHistogram {
_, err = app.AppendHistogram(0, lset, t, &histogramSample, nil)
}
Expand Down
20 changes: 20 additions & 0 deletions test/e2e/e2ethanos/services.go
Original file line number Diff line number Diff line change
Expand Up @@ -260,6 +260,10 @@ type QuerierBuilder struct {
replicaLabels []string
tracingConfig string

telemetryDurationQuantiles []float64
telemetrySamplesQuantiles []float64
telemetrySeriesQuantiles []float64

e2e.Linkable
f e2e.FutureRunnable
}
Expand Down Expand Up @@ -369,6 +373,13 @@ func (q *QuerierBuilder) WithQueryMode(mode string) *QuerierBuilder {
return q
}

func (q *QuerierBuilder) WithTelemetryQuantiles(duration []float64, samples []float64, series []float64) *QuerierBuilder {
q.telemetryDurationQuantiles = duration
q.telemetrySamplesQuantiles = samples
q.telemetrySeriesQuantiles = series
return q
}

func (q *QuerierBuilder) Init() *e2emon.InstrumentedRunnable {
args, err := q.collectArgs()
if err != nil {
Expand Down Expand Up @@ -459,6 +470,15 @@ func (q *QuerierBuilder) collectArgs() ([]string, error) {
if q.tracingConfig != "" {
args = append(args, "--tracing.config="+q.tracingConfig)
}
for _, bucket := range q.telemetryDurationQuantiles {
args = append(args, "--query.telemetry.request-duration-seconds-quantiles="+strconv.FormatFloat(bucket, 'f', -1, 64))
}
for _, bucket := range q.telemetrySamplesQuantiles {
args = append(args, "--query.telemetry.request-samples-quantiles="+strconv.FormatFloat(bucket, 'f', -1, 64))
}
for _, bucket := range q.telemetrySeriesQuantiles {
args = append(args, "--query.telemetry.request-series-seconds-quantiles="+strconv.FormatFloat(bucket, 'f', -1, 64))
}
return args, nil
}

Expand Down
35 changes: 32 additions & 3 deletions test/e2e/query_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -598,7 +598,7 @@ func TestQueryStoreMetrics(t *testing.T) {
bkt, err := s3.NewBucketWithConfig(l, e2ethanos.NewS3Config(bucket, minio.Endpoint("http"), minio.Dir()), "test")
testutil.Ok(t, err)

// Preparing 2 different blocks for the tests.
// Preparing 3 different blocks for the tests.
{
blockSizes := []struct {
samples int
Expand All @@ -607,6 +607,7 @@ func TestQueryStoreMetrics(t *testing.T) {
}{
{samples: 10, series: 1, name: "one_series"},
{samples: 10, series: 1001, name: "thousand_one_series"},
{samples: 10, series: 10001, name: "inf_series"},
}
now := time.Now()
externalLabels := labels.FromStrings("prometheus", "p1", "replica", "0")
Expand Down Expand Up @@ -644,9 +645,12 @@ func TestQueryStoreMetrics(t *testing.T) {
"",
nil,
)
querier := e2ethanos.NewQuerierBuilder(e, "1", storeGW.InternalEndpoint("grpc")).Init()

sampleBuckets := []float64{100, 1000, 10000, 100000}
seriesBuckets := []float64{10, 100, 1000, 10000}
querier := e2ethanos.NewQuerierBuilder(e, "1", storeGW.InternalEndpoint("grpc")).WithTelemetryQuantiles(nil, sampleBuckets, seriesBuckets).Init()
testutil.Ok(t, e2e.StartAndWaitReady(storeGW, querier))
testutil.Ok(t, storeGW.WaitSumMetrics(e2emon.Equals(2), "thanos_blocks_meta_synced"))
testutil.Ok(t, storeGW.WaitSumMetrics(e2emon.Equals(3), "thanos_blocks_meta_synced"))

// Querying the series in the previously created blocks to ensure we produce Store API query metrics.
{
Expand All @@ -663,6 +667,13 @@ func TestQueryStoreMetrics(t *testing.T) {
Deduplicate: true,
}, 1001)
testutil.Ok(t, err)

instantQuery(t, ctx, querier.Endpoint("http"), func() string {
return "max_over_time(inf_series[2h])"
}, time.Now, promclient.QueryOptions{
Deduplicate: true,
}, 10001)
testutil.Ok(t, err)
}

mon, err := e2emon.Start(e)
Expand Down Expand Up @@ -701,6 +712,24 @@ func TestQueryStoreMetrics(t *testing.T) {
Value: model.SampleValue(1),
},
})

queryWaitAndAssert(t, ctx, mon.GetMonitoringRunnable().Endpoint(e2edb.AccessPortName), func() string {
return "thanos_store_api_query_duration_seconds_count{samples_le='+Inf',series_le='+Inf'}"
}, time.Now, promclient.QueryOptions{
Deduplicate: true,
}, model.Vector{
&model.Sample{
Metric: model.Metric{
"__name__": "thanos_store_api_query_duration_seconds_count",
"instance": "storemetrics01-querier-1:8080",
"job": "querier-1",
"samples_le": "+Inf",
"series_le": "+Inf",
},
Value: model.SampleValue(1),
},
})

}

// Regression test for https://github.com/thanos-io/thanos/issues/5033.
Expand Down

0 comments on commit f6dfd7f

Please sign in to comment.