Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/logql/metrics.go

333 lines
12 KiB

package logql
import (
"context"
"hash/fnv"
"strings"
"time"
"github.com/dustin/go-humanize"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
promql_parser "github.com/prometheus/prometheus/promql/parser"
"github.com/grafana/loki/pkg/analytics"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel"
logql_stats "github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/util/httpreq"
util_log "github.com/grafana/loki/pkg/util/log"
)
const (
QueryTypeMetric = "metric"
QueryTypeFilter = "filter"
QueryTypeLimited = "limited"
QueryTypeLabels = "labels"
QueryTypeSeries = "series"
latencyTypeSlow = "slow"
latencyTypeFast = "fast"
slowQueryThresholdSecond = float64(10)
)
var (
bytesPerSecond = promauto.NewHistogramVec(prometheus.HistogramOpts{
Namespace: "loki",
Name: "logql_querystats_bytes_processed_per_seconds",
Help: "Distribution of bytes processed per second for LogQL queries.",
Feature/querysharding ii (#1927) * [wip] sharding evaluator/ast * [wip] continues experimenting with ast mapping * refactoring in preparation for binops * evaluators can pass state to other evaluators * compiler alignment * Evaluator method renamed to StepEvaluator * chained evaluator impl * tidying up sharding code * handling for ConcatSampleExpr * downstream iterator * structure for downstreaming asts * outlines sharding optimizations * work on sharding mapper * ast sharding optimizations * test for different logrange positions * shard mapper tests * stronger ast sharding & tests * shardmapper tests for string->string * removes sharding evaluator code * removes unused ctx arg * Revert "removes sharding evaluator code" This reverts commit 55d41b9519da9496e9471f13a5048d903ea04aaa. * interfaces for downstreaming, type conversions * sharding plumbing on frontend * type alignment in queryrange to downstream sharded queriers * downstreaming support for sharding incl storage code * removes chainedevaluator * comment alignment * storage shard injection * speccing out testware for sharding equivalence * [wip] shared engine refactor * sorting streams, sharding eval fixes * downstream evaluator embeds defaultevaluator * other pkgs adopt logql changes * metrics & logs use same middleware instantiation process * wires up shardingware * middleware per metrics/logfilter * empty step populating StepEvaluator promql.Matrix adapter * sharding metrics * log/span injection into sharded engine * sharding metrics avoids multiple instantiation * downstreamhandler tracing * sharding parameterized libsonnet * removes querier replicas * default 32 concurrency for workers * jsonnet correct level override * unquote true in yaml * lowercase error + downstreamEvaluator defaults to embedded defaultEvaluator * makes shardRecorder private * logs query on failed parse * refactors engine to be multi-use, minimizes logger injection, generalizes Query methods, removes Engine interface * basic tests for querysharding mware * [wip] concurrent evaluator * integrates stat propagation into sharding evaluator * splitby histogram * extends le bounds for bytes processed * byte throughput histogram buckets to 40gb * chunk duration mixin * fixes merge w/ field rename * derives logger in sharded engine via ctx & logs some downstream evaluators * moves sharded engine to top, adds comments * logs failed merge results in stats ctx * snapshotting stats merge logic is done more effectively * per query concurrency controlled via downstreamer * unexports decodereq * queryrange testware * downstreamer tests * pr requests
6 years ago
// 50MB 100MB 200MB 400MB 600MB 800MB 1GB 2GB 3GB 4GB 5GB 6GB 7GB 8GB 9GB 10GB 15GB 20GB 30GB, 40GB 50GB 60GB
Buckets: []float64{50 * 1e6, 100 * 1e6, 400 * 1e6, 600 * 1e6, 800 * 1e6, 1 * 1e9, 2 * 1e9, 3 * 1e9, 4 * 1e9, 5 * 1e9, 6 * 1e9, 7 * 1e9, 8 * 1e9, 9 * 1e9, 10 * 1e9, 15 * 1e9, 20 * 1e9, 30 * 1e9, 40 * 1e9, 50 * 1e9, 60 * 1e9},
}, []string{"status_code", "type", "range", "latency_type"})
execLatency = promauto.NewHistogramVec(prometheus.HistogramOpts{
Namespace: "loki",
Name: "logql_querystats_latency_seconds",
Help: "Distribution of latency for LogQL queries.",
// 0.25 0.5 1 2 4 8 16 32 64 128
Buckets: prometheus.ExponentialBuckets(0.250, 2, 10),
}, []string{"status_code", "type", "range"})
chunkDownloadLatency = promauto.NewHistogramVec(prometheus.HistogramOpts{
Namespace: "loki",
Name: "logql_querystats_chunk_download_latency_seconds",
Help: "Distribution of chunk downloads latency for LogQL queries.",
// 0.25 0.5 1 2 4 8 16 32 64 128
Buckets: prometheus.ExponentialBuckets(0.250, 2, 10),
}, []string{"status_code", "type", "range"})
duplicatesTotal = promauto.NewCounter(prometheus.CounterOpts{
Namespace: "loki",
Name: "logql_querystats_duplicates_total",
Help: "Total count of duplicates found while executing LogQL queries.",
})
chunkDownloadedTotal = promauto.NewCounterVec(prometheus.CounterOpts{
Namespace: "loki",
Name: "logql_querystats_downloaded_chunk_total",
Help: "Total count of chunks downloaded found while executing LogQL queries.",
}, []string{"status_code", "type", "range"})
ingesterLineTotal = promauto.NewCounter(prometheus.CounterOpts{
Namespace: "loki",
Name: "logql_querystats_ingester_sent_lines_total",
Help: "Total count of lines sent from ingesters while executing LogQL queries.",
})
bytePerSecondMetricUsage = analytics.NewStatistics("query_metric_bytes_per_second")
bytePerSecondLogUsage = analytics.NewStatistics("query_log_bytes_per_second")
linePerSecondMetricUsage = analytics.NewStatistics("query_metric_lines_per_second")
linePerSecondLogUsage = analytics.NewStatistics("query_log_lines_per_second")
)
func RecordRangeAndInstantQueryMetrics(
ctx context.Context,
log log.Logger,
p Params,
status string,
stats logql_stats.Result,
result promql_parser.Value,
) {
var (
logger = util_log.WithContext(ctx, log)
rt = string(GetRangeType(p))
latencyType = latencyTypeFast
returnedLines = 0
)
queryType, err := QueryType(p.Query())
if err != nil {
level.Warn(logger).Log("msg", "error parsing query type", "err", err)
}
// Tag throughput metric by latency type based on a threshold.
// Latency below the threshold is fast, above is slow.
if stats.Summary.ExecTime > slowQueryThresholdSecond {
latencyType = latencyTypeSlow
}
if result != nil && result.Type() == logqlmodel.ValueTypeStreams {
returnedLines = int(result.(logqlmodel.Streams).Lines())
}
queryTags, _ := ctx.Value(httpreq.QueryTagsHTTPHeader).(string) // it's ok to be empty.
logValues := make([]interface{}, 0, 30)
logValues = append(logValues, []interface{}{
"latency", latencyType, // this can be used to filter log lines.
"query", p.Query(),
"query_hash", HashedQuery(p.Query()),
"query_type", queryType,
"range_type", rt,
"length", p.End().Sub(p.Start()),
"start_delta", time.Since(p.Start()),
"end_delta", time.Since(p.End()),
"step", p.Step(),
"duration", logql_stats.ConvertSecondsToNanoseconds(stats.Summary.ExecTime),
"status", status,
"limit", p.Limit(),
"returned_lines", returnedLines,
"throughput", strings.Replace(humanize.Bytes(uint64(stats.Summary.BytesProcessedPerSecond)), " ", "", 1),
"total_bytes", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalBytesProcessed)), " ", "", 1),
"lines_per_second", stats.Summary.LinesProcessedPerSecond,
"total_lines", stats.Summary.TotalLinesProcessed,
"total_entries", stats.Summary.TotalEntriesReturned,
"store_chunks_download_time", stats.ChunksDownloadTime(),
"queue_time", logql_stats.ConvertSecondsToNanoseconds(stats.Summary.QueueTime),
"splits", stats.Summary.Splits,
"shards", stats.Summary.Shards,
"cache_chunk_req", stats.Caches.Chunk.EntriesRequested,
"cache_chunk_hit", stats.Caches.Chunk.EntriesFound,
"cache_chunk_bytes_stored", stats.Caches.Chunk.BytesSent,
"cache_chunk_bytes_fetched", stats.Caches.Chunk.BytesReceived,
"cache_chunk_download_time", stats.Caches.Chunk.CacheDownloadTime(),
"cache_index_req", stats.Caches.Index.EntriesRequested,
"cache_index_hit", stats.Caches.Index.EntriesFound,
"cache_index_download_time", stats.Caches.Index.CacheDownloadTime(),
Add back cache stats for index stats requests (#9816) **What this PR does / why we need it**: In https://github.com/grafana/loki/pull/9536, we added cache stats for index stats requests. That PR had a bug that inflated the query stats due to reusing the stats context in the query engine. Therefore, we had to revert the PR at https://github.com/grafana/loki/pull/9721. This PR brings back the changes from https://github.com/grafana/loki/pull/9536 but fixes the inflated starts by no longer reusing the same context in the query engine, but rather creating a new one for the shard resolver. I tested it on a dev cluster and seems to be working fine. here's the output for the same query: **Stats with the bug from #9536**: ``` ... Cache.StatsResult.Requests 980 Cache.StatsResult.EntriesRequested 490 Cache.StatsResult.EntriesFound 0 Cache.StatsResult.EntriesStored 490 Cache.StatsResult.BytesSent 0 B Cache.StatsResult.BytesReceived 0 B ... Summary.BytesProcessedPerSecond 43 GB Summary.LinesProcessedPerSecond 93305142 Summary.TotalBytesProcessed 945 GB Summary.TotalLinesProcessed 2059694183 ``` **Stats from _main_** ``` ... Summary.BytesProcessedPerSecond 1.6 GB Summary.LinesProcessedPerSecond 3403718 Summary.TotalBytesProcessed 95 GB Summary.TotalLinesProcessed 207971404 ``` **Stats with fix in this PR** ``` .. Cache.StatsResult.Requests 132 Cache.StatsResult.EntriesRequested 66 Cache.StatsResult.EntriesFound 0 Cache.StatsResult.EntriesStored 66 Cache.StatsResult.BytesSent 0 B Cache.StatsResult.BytesReceived 0 B ... Summary.BytesProcessedPerSecond 4.3 GB Summary.LinesProcessedPerSecond 9468900 Summary.TotalBytesProcessed 95 GB Summary.TotalLinesProcessed 207793816 ``` As can be seen, with the changes in this PR, the summary stats are no longer inflated. **Which issue(s) this PR fixes**: Fixes https://github.com/grafana/loki/pull/9536 **Special notes for your reviewer**: I think it's ok to skip reviewing the changes from the commit cherry-picking the changes from https://github.com/grafana/loki/pull/9536 **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/upgrading/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
2 years ago
"cache_stats_results_req", stats.Caches.StatsResult.EntriesRequested,
"cache_stats_results_hit", stats.Caches.StatsResult.EntriesFound,
"cache_stats_results_download_time", stats.Caches.StatsResult.CacheDownloadTime(),
"cache_result_req", stats.Caches.Result.EntriesRequested,
"cache_result_hit", stats.Caches.Result.EntriesFound,
"cache_result_download_time", stats.Caches.Result.CacheDownloadTime(),
}...)
logValues = append(logValues, tagsToKeyValues(queryTags)...)
level.Info(logger).Log(
logValues...,
)
bytesPerSecond.WithLabelValues(status, queryType, rt, latencyType).
Observe(float64(stats.Summary.BytesProcessedPerSecond))
execLatency.WithLabelValues(status, queryType, rt).
Observe(stats.Summary.ExecTime)
chunkDownloadLatency.WithLabelValues(status, queryType, rt).
Observe(stats.ChunksDownloadTime().Seconds())
duplicatesTotal.Add(float64(stats.TotalDuplicates()))
chunkDownloadedTotal.WithLabelValues(status, queryType, rt).
Add(float64(stats.TotalChunksDownloaded()))
ingesterLineTotal.Add(float64(stats.Ingester.TotalLinesSent))
recordUsageStats(queryType, stats)
}
func HashedQuery(query string) uint32 {
h := fnv.New32()
_, _ = h.Write([]byte(query))
return h.Sum32()
}
func RecordLabelQueryMetrics(
ctx context.Context,
log log.Logger,
start, end time.Time,
label, query, status string,
stats logql_stats.Result,
) {
var (
logger = util_log.WithContext(ctx, log)
latencyType = latencyTypeFast
queryType = QueryTypeLabels
)
// Tag throughput metric by latency type based on a threshold.
// Latency below the threshold is fast, above is slow.
if stats.Summary.ExecTime > slowQueryThresholdSecond {
latencyType = latencyTypeSlow
}
level.Info(logger).Log(
"latency", latencyType,
"query_type", queryType,
"length", end.Sub(start),
"duration", time.Duration(int64(stats.Summary.ExecTime*float64(time.Second))),
"status", status,
"label", label,
"query", query,
"splits", stats.Summary.Splits,
"throughput", strings.Replace(humanize.Bytes(uint64(stats.Summary.BytesProcessedPerSecond)), " ", "", 1),
"total_bytes", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalBytesProcessed)), " ", "", 1),
"total_entries", stats.Summary.TotalEntriesReturned,
)
bytesPerSecond.WithLabelValues(status, queryType, "", latencyType).
Observe(float64(stats.Summary.BytesProcessedPerSecond))
execLatency.WithLabelValues(status, queryType, "").
Observe(stats.Summary.ExecTime)
chunkDownloadLatency.WithLabelValues(status, queryType, "").
Observe(stats.ChunksDownloadTime().Seconds())
duplicatesTotal.Add(float64(stats.TotalDuplicates()))
chunkDownloadedTotal.WithLabelValues(status, queryType, "").
Add(float64(stats.TotalChunksDownloaded()))
ingesterLineTotal.Add(float64(stats.Ingester.TotalLinesSent))
}
func PrintMatches(matches []string) string {
// not using comma (,) as separator as matcher may already have comma (e.g: `{a="b", c="d"}`)
return strings.Join(matches, ":")
}
func RecordSeriesQueryMetrics(
ctx context.Context,
log log.Logger,
start, end time.Time,
match []string,
status string,
stats logql_stats.Result,
) {
var (
logger = util_log.WithContext(ctx, log)
latencyType = latencyTypeFast
queryType = QueryTypeSeries
)
// Tag throughput metric by latency type based on a threshold.
// Latency below the threshold is fast, above is slow.
if stats.Summary.ExecTime > slowQueryThresholdSecond {
latencyType = latencyTypeSlow
}
// we also log queries, useful for troubleshooting slow queries.
level.Info(logger).Log(
"latency", latencyType,
"query_type", queryType,
"length", end.Sub(start),
"duration", time.Duration(int64(stats.Summary.ExecTime*float64(time.Second))),
"status", status,
"match", PrintMatches(match),
"splits", stats.Summary.Splits,
"throughput", strings.Replace(humanize.Bytes(uint64(stats.Summary.BytesProcessedPerSecond)), " ", "", 1),
"total_bytes", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalBytesProcessed)), " ", "", 1),
"total_entries", stats.Summary.TotalEntriesReturned,
)
bytesPerSecond.WithLabelValues(status, queryType, "", latencyType).
Observe(float64(stats.Summary.BytesProcessedPerSecond))
execLatency.WithLabelValues(status, queryType, "").
Observe(stats.Summary.ExecTime)
chunkDownloadLatency.WithLabelValues(status, queryType, "").
Observe(stats.ChunksDownloadTime().Seconds())
duplicatesTotal.Add(float64(stats.TotalDuplicates()))
chunkDownloadedTotal.WithLabelValues(status, queryType, "").
Add(float64(stats.TotalChunksDownloaded()))
ingesterLineTotal.Add(float64(stats.Ingester.TotalLinesSent))
}
func recordUsageStats(queryType string, stats logql_stats.Result) {
if queryType == QueryTypeMetric {
bytePerSecondMetricUsage.Record(float64(stats.Summary.BytesProcessedPerSecond))
linePerSecondMetricUsage.Record(float64(stats.Summary.LinesProcessedPerSecond))
} else {
bytePerSecondLogUsage.Record(float64(stats.Summary.BytesProcessedPerSecond))
linePerSecondLogUsage.Record(float64(stats.Summary.LinesProcessedPerSecond))
}
}
func QueryType(query string) (string, error) {
expr, err := syntax.ParseExpr(query)
if err != nil {
return "", err
}
switch e := expr.(type) {
case syntax.SampleExpr:
return QueryTypeMetric, nil
case syntax.LogSelectorExpr:
if e.HasFilter() {
return QueryTypeFilter, nil
}
return QueryTypeLimited, nil
default:
return "", nil
}
}
// tagsToKeyValues converts QueryTags to form that is easy to log.
// e.g: `Source=foo,Feature=beta` -> []interface{}{"source", "foo", "feature", "beta"}
// so that we could log nicely!
// If queryTags is not in canonical form then its completely ignored (e.g: `key1=value1,key2=value`)
func tagsToKeyValues(queryTags string) []interface{} {
toks := strings.FieldsFunc(queryTags, func(r rune) bool {
return r == ','
})
vals := make([]string, 0)
for _, tok := range toks {
val := strings.FieldsFunc(tok, func(r rune) bool {
return r == '='
})
if len(val) != 2 {
continue
}
vals = append(vals, val...)
}
res := make([]interface{}, 0, len(vals))
for _, val := range vals {
res = append(res, strings.ToLower(val))
}
return res
}