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/querier/queryrange/log_result_cache.go

333 lines
11 KiB

package queryrange
import (
"context"
"fmt"
"net/http"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/gogo/protobuf/proto"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"github.com/weaveworks/common/httpgrpc"
"golang.org/x/sync/errgroup"
"github.com/grafana/dskit/tenant"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/util/validation"
)
// LogResultCacheMetrics is the metrics wrapper used in log result cache.
type LogResultCacheMetrics struct {
CacheHit prometheus.Counter
CacheMiss prometheus.Counter
}
// NewLogResultCacheMetrics creates metrics to be used in log result cache.
func NewLogResultCacheMetrics(registerer prometheus.Registerer) *LogResultCacheMetrics {
return &LogResultCacheMetrics{
CacheHit: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Namespace: "loki",
Name: "query_frontend_log_result_cache_hit_total",
}),
CacheMiss: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Namespace: "loki",
Name: "query_frontend_log_result_cache_miss_total",
}),
}
}
// NewLogResultCache creates a new log result cache middleware.
// Currently it only caches empty filter queries, this is because those are usually easily and freely cacheable.
// Log hits are difficult to handle because of the limit query parameter and the size of the response.
// In the future it could be extended to cache non-empty query results.
// see https://docs.google.com/document/d/1_mACOpxdWZ5K0cIedaja5gzMbv-m0lUVazqZd2O4mEU/edit
func NewLogResultCache(logger log.Logger, limits Limits, cache cache.Cache, shouldCache queryrangebase.ShouldCacheFn,
transformer UserIDTransformer, metrics *LogResultCacheMetrics) queryrangebase.Middleware {
if metrics == nil {
metrics = NewLogResultCacheMetrics(nil)
}
return queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler {
return &logResultCache{
next: next,
limits: limits,
cache: cache,
logger: logger,
shouldCache: shouldCache,
transformer: transformer,
metrics: metrics,
}
})
}
type logResultCache struct {
next queryrangebase.Handler
limits Limits
cache cache.Cache
shouldCache queryrangebase.ShouldCacheFn
transformer UserIDTransformer
metrics *LogResultCacheMetrics
logger log.Logger
}
func (l *logResultCache) Do(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
tenantIDs, err := tenant.TenantIDs(ctx)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
if l.shouldCache != nil && !l.shouldCache(req) {
return l.next.Do(ctx, req)
}
cacheFreshnessCapture := func(id string) time.Duration { return l.limits.MaxCacheFreshness(ctx, id) }
maxCacheFreshness := validation.MaxDurationPerTenant(tenantIDs, cacheFreshnessCapture)
maxCacheTime := int64(model.Now().Add(-maxCacheFreshness))
if req.GetEnd() > maxCacheTime {
return l.next.Do(ctx, req)
}
lokiReq, ok := req.(*LokiRequest)
if !ok {
return nil, httpgrpc.Errorf(http.StatusInternalServerError, "invalid request type %T", req)
}
interval := validation.SmallestPositiveNonZeroDurationPerTenant(tenantIDs, l.limits.QuerySplitDuration)
// skip caching by if interval is unset
if interval == 0 {
return l.next.Do(ctx, req)
}
// The first subquery might not be aligned.
alignedStart := time.Unix(0, lokiReq.GetStartTs().UnixNano()-(lokiReq.GetStartTs().UnixNano()%interval.Nanoseconds()))
// generate the cache key based on query, tenant and start time.
transformedTenantIDs := tenantIDs
if l.transformer != nil {
transformedTenantIDs = make([]string, 0, len(tenantIDs))
for _, tenantID := range tenantIDs {
transformedTenantIDs = append(transformedTenantIDs, l.transformer(ctx, tenantID))
}
}
cacheKey := fmt.Sprintf("log:%s:%s:%d:%d", tenant.JoinTenantIDs(transformedTenantIDs), req.GetQuery(), interval.Nanoseconds(), alignedStart.UnixNano()/(interval.Nanoseconds()))
_, buff, _, err := l.cache.Fetch(ctx, []string{cache.HashKey(cacheKey)})
if err != nil {
level.Warn(l.logger).Log("msg", "error fetching cache", "err", err, "cacheKey", cacheKey)
return l.next.Do(ctx, req)
}
// we expect only one key to be found or missing.
if len(buff) > 1 {
level.Warn(l.logger).Log("msg", "unexpected length of cache return values", "buff", len(buff))
return l.next.Do(ctx, req)
}
if len(buff) == 0 {
// cache miss
return l.handleMiss(ctx, cacheKey, lokiReq)
}
// cache hit
var cachedRequest LokiRequest
err = proto.Unmarshal(buff[0], &cachedRequest)
if err != nil {
level.Warn(l.logger).Log("msg", "error unmarshalling request from cache", "err", err)
return l.next.Do(ctx, req)
}
return l.handleHit(ctx, cacheKey, &cachedRequest, lokiReq)
}
func (l *logResultCache) handleMiss(ctx context.Context, cacheKey string, req *LokiRequest) (queryrangebase.Response, error) {
l.metrics.CacheMiss.Inc()
level.Debug(l.logger).Log("msg", "cache miss", "key", cacheKey)
resp, err := l.next.Do(ctx, req)
if err != nil {
return nil, err
}
lokiRes, ok := resp.(*LokiResponse)
if !ok {
return nil, fmt.Errorf("unexpected response type %T", resp)
}
// At the moment we only cache empty results
if !isEmpty(lokiRes) {
return resp, nil
}
data, err := proto.Marshal(req)
if err != nil {
level.Warn(l.logger).Log("msg", "error marshalling request", "err", err)
return resp, nil
}
// cache the result
err = l.cache.Store(ctx, []string{cache.HashKey(cacheKey)}, [][]byte{data})
if err != nil {
level.Warn(l.logger).Log("msg", "error storing cache", "err", err)
}
return resp, nil
}
func (l *logResultCache) handleHit(ctx context.Context, cacheKey string, cachedRequest *LokiRequest, lokiReq *LokiRequest) (queryrangebase.Response, error) {
l.metrics.CacheHit.Inc()
// we start with an empty response
result := emptyResponse(cachedRequest)
// if the request is the same and cover the whole time range,
// we can just return the cached result.
fix bugs in logs results caching and its tests (#7925) **What this PR does / why we need it**: When a logs query results in an empty response, we cache it to avoid doing that query again and respond straight away with an empty response. However, we cache a single entry per time split interval with the query itself to keep things simple. For example, if the time split config for tenant `A` is `30m`, then queries for intervals `10m`-`20m` and `21m`-`25m` would have the same cache key. Here is roughly how cache hit is handled: * If the new query is within the cached query bounds, return empty results * If the start of new query is before the start time of the cached query, do a query from `newQuery.Start` to `cachedQuery.Start` * If the response of last query is also empty, set `cachedQuery.Start` = `newQuery.Start` * If the end of new query is after the end time of the cached query, do a query from `cachedQuery.End` to `newQuery.Start` * If the response of last query is also empty, set `cachedQuery.End` = `newQuery.End` * If we have changes in `cachedQuery.Start/End`, update it in the cache. The problem here is when we do queries to fill the gap, we sometimes do queries for the range outside of what the user requested and respond back without reducing the response to what the user requested. For example, if the cached query is from `21m`-`25m` and the user query is from `10m`-`15m`, we will query for the whole gap i.e `10m`-`21m`. If there are logs from `15m`-`21m` in the response, we will unexpectedly send it back to the user. This PR takes care of this issue by extracting the data and sending back only the user's requested logs. I have also found the tests for logs results cache were incorrect. They heavily use [mergeLokiResponse](https://github.com/grafana/loki/blob/e2842c69c5e089389b697faa792e368f42f84c19/pkg/querier/queryrange/codec.go#L1014) for building the test data, but they were not setting the `limit` in test queries which was causing `mergeLokiResponse` to send back empty results. This means were were always comparing two empty results, which would always be the same. This PR also takes care of fixing it, and relevant changes to correct the test. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated
3 years ago
if cachedRequest.StartTs.UnixNano() <= lokiReq.StartTs.UnixNano() && cachedRequest.EndTs.UnixNano() >= lokiReq.EndTs.UnixNano() {
return result, nil
}
// we could be missing data at the start and the end.
// so we're going to fetch what is missing.
var (
startRequest, endRequest *LokiRequest
startResp, endResp *LokiResponse
updateCache bool
ok bool
)
g, ctx := errgroup.WithContext(ctx)
// if we're missing data at the start, start fetching from the start to the cached start.
if lokiReq.GetStartTs().Before(cachedRequest.GetStartTs()) {
g.Go(func() error {
startRequest = lokiReq.WithStartEndTime(lokiReq.GetStartTs(), cachedRequest.GetStartTs())
resp, err := l.next.Do(ctx, startRequest)
if err != nil {
return err
}
startResp, ok = resp.(*LokiResponse)
if !ok {
return fmt.Errorf("unexpected response type %T", resp)
}
return nil
})
}
// if we're missing data at the end, start fetching from the cached end to the end.
if lokiReq.GetEndTs().After(cachedRequest.GetEndTs()) {
g.Go(func() error {
endRequest = lokiReq.WithStartEndTime(cachedRequest.GetEndTs(), lokiReq.GetEndTs())
resp, err := l.next.Do(ctx, endRequest)
if err != nil {
return err
}
endResp, ok = resp.(*LokiResponse)
if !ok {
return fmt.Errorf("unexpected response type %T", resp)
}
return nil
})
}
if err := g.Wait(); err != nil {
return nil, err
}
// if we have data at the start, we need to merge it with the cached data if it's empty and update the cache.
// If it's not empty only merge the response.
if startResp != nil {
if isEmpty(startResp) {
cachedRequest = cachedRequest.WithStartEndTime(startRequest.GetStartTs(), cachedRequest.GetEndTs())
updateCache = true
} else {
if startResp.Status != loghttp.QueryStatusSuccess {
return startResp, nil
}
fix bugs in logs results caching and its tests (#7925) **What this PR does / why we need it**: When a logs query results in an empty response, we cache it to avoid doing that query again and respond straight away with an empty response. However, we cache a single entry per time split interval with the query itself to keep things simple. For example, if the time split config for tenant `A` is `30m`, then queries for intervals `10m`-`20m` and `21m`-`25m` would have the same cache key. Here is roughly how cache hit is handled: * If the new query is within the cached query bounds, return empty results * If the start of new query is before the start time of the cached query, do a query from `newQuery.Start` to `cachedQuery.Start` * If the response of last query is also empty, set `cachedQuery.Start` = `newQuery.Start` * If the end of new query is after the end time of the cached query, do a query from `cachedQuery.End` to `newQuery.Start` * If the response of last query is also empty, set `cachedQuery.End` = `newQuery.End` * If we have changes in `cachedQuery.Start/End`, update it in the cache. The problem here is when we do queries to fill the gap, we sometimes do queries for the range outside of what the user requested and respond back without reducing the response to what the user requested. For example, if the cached query is from `21m`-`25m` and the user query is from `10m`-`15m`, we will query for the whole gap i.e `10m`-`21m`. If there are logs from `15m`-`21m` in the response, we will unexpectedly send it back to the user. This PR takes care of this issue by extracting the data and sending back only the user's requested logs. I have also found the tests for logs results cache were incorrect. They heavily use [mergeLokiResponse](https://github.com/grafana/loki/blob/e2842c69c5e089389b697faa792e368f42f84c19/pkg/querier/queryrange/codec.go#L1014) for building the test data, but they were not setting the `limit` in test queries which was causing `mergeLokiResponse` to send back empty results. This means were were always comparing two empty results, which would always be the same. This PR also takes care of fixing it, and relevant changes to correct the test. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated
3 years ago
result = mergeLokiResponse(extractLokiResponse(lokiReq.GetStartTs(), lokiReq.GetEndTs(), startResp), result)
}
}
// if we have data at the end, we need to merge it with the cached data if it's empty and update the cache.
// If it's not empty only merge the response.
if endResp != nil {
if isEmpty(endResp) {
cachedRequest = cachedRequest.WithStartEndTime(cachedRequest.GetStartTs(), endRequest.GetEndTs())
updateCache = true
} else {
if endResp.Status != loghttp.QueryStatusSuccess {
return endResp, nil
}
fix bugs in logs results caching and its tests (#7925) **What this PR does / why we need it**: When a logs query results in an empty response, we cache it to avoid doing that query again and respond straight away with an empty response. However, we cache a single entry per time split interval with the query itself to keep things simple. For example, if the time split config for tenant `A` is `30m`, then queries for intervals `10m`-`20m` and `21m`-`25m` would have the same cache key. Here is roughly how cache hit is handled: * If the new query is within the cached query bounds, return empty results * If the start of new query is before the start time of the cached query, do a query from `newQuery.Start` to `cachedQuery.Start` * If the response of last query is also empty, set `cachedQuery.Start` = `newQuery.Start` * If the end of new query is after the end time of the cached query, do a query from `cachedQuery.End` to `newQuery.Start` * If the response of last query is also empty, set `cachedQuery.End` = `newQuery.End` * If we have changes in `cachedQuery.Start/End`, update it in the cache. The problem here is when we do queries to fill the gap, we sometimes do queries for the range outside of what the user requested and respond back without reducing the response to what the user requested. For example, if the cached query is from `21m`-`25m` and the user query is from `10m`-`15m`, we will query for the whole gap i.e `10m`-`21m`. If there are logs from `15m`-`21m` in the response, we will unexpectedly send it back to the user. This PR takes care of this issue by extracting the data and sending back only the user's requested logs. I have also found the tests for logs results cache were incorrect. They heavily use [mergeLokiResponse](https://github.com/grafana/loki/blob/e2842c69c5e089389b697faa792e368f42f84c19/pkg/querier/queryrange/codec.go#L1014) for building the test data, but they were not setting the `limit` in test queries which was causing `mergeLokiResponse` to send back empty results. This means were were always comparing two empty results, which would always be the same. This PR also takes care of fixing it, and relevant changes to correct the test. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated
3 years ago
result = mergeLokiResponse(extractLokiResponse(lokiReq.GetStartTs(), lokiReq.GetEndTs(), endResp), result)
}
}
// we need to update the cache since we fetched more either at the end or the start and it was empty.
if updateCache {
data, err := proto.Marshal(cachedRequest)
if err != nil {
level.Warn(l.logger).Log("msg", "error marshalling request", "err", err)
return result, err
}
// cache the result
err = l.cache.Store(ctx, []string{cache.HashKey(cacheKey)}, [][]byte{data})
if err != nil {
level.Warn(l.logger).Log("msg", "error storing cache", "err", err)
}
}
return result, nil
}
fix bugs in logs results caching and its tests (#7925) **What this PR does / why we need it**: When a logs query results in an empty response, we cache it to avoid doing that query again and respond straight away with an empty response. However, we cache a single entry per time split interval with the query itself to keep things simple. For example, if the time split config for tenant `A` is `30m`, then queries for intervals `10m`-`20m` and `21m`-`25m` would have the same cache key. Here is roughly how cache hit is handled: * If the new query is within the cached query bounds, return empty results * If the start of new query is before the start time of the cached query, do a query from `newQuery.Start` to `cachedQuery.Start` * If the response of last query is also empty, set `cachedQuery.Start` = `newQuery.Start` * If the end of new query is after the end time of the cached query, do a query from `cachedQuery.End` to `newQuery.Start` * If the response of last query is also empty, set `cachedQuery.End` = `newQuery.End` * If we have changes in `cachedQuery.Start/End`, update it in the cache. The problem here is when we do queries to fill the gap, we sometimes do queries for the range outside of what the user requested and respond back without reducing the response to what the user requested. For example, if the cached query is from `21m`-`25m` and the user query is from `10m`-`15m`, we will query for the whole gap i.e `10m`-`21m`. If there are logs from `15m`-`21m` in the response, we will unexpectedly send it back to the user. This PR takes care of this issue by extracting the data and sending back only the user's requested logs. I have also found the tests for logs results cache were incorrect. They heavily use [mergeLokiResponse](https://github.com/grafana/loki/blob/e2842c69c5e089389b697faa792e368f42f84c19/pkg/querier/queryrange/codec.go#L1014) for building the test data, but they were not setting the `limit` in test queries which was causing `mergeLokiResponse` to send back empty results. This means were were always comparing two empty results, which would always be the same. This PR also takes care of fixing it, and relevant changes to correct the test. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated
3 years ago
// extractLokiResponse extracts response with interval [start, end)
func extractLokiResponse(start, end time.Time, r *LokiResponse) *LokiResponse {
extractedResp := LokiResponse{
Status: r.Status,
Direction: r.Direction,
Limit: r.Limit,
Version: r.Version,
ErrorType: r.ErrorType,
Error: r.Error,
Statistics: r.Statistics,
Data: LokiData{
ResultType: r.Data.ResultType,
Result: []logproto.Stream{},
},
}
for _, stream := range r.Data.Result {
if stream.Entries[0].Timestamp.After(end) || stream.Entries[len(stream.Entries)-1].Timestamp.Before(start) {
continue
}
extractedStream := logproto.Stream{
Labels: stream.Labels,
Entries: []logproto.Entry{},
Hash: stream.Hash,
}
for _, entry := range stream.Entries {
if entry.Timestamp.Before(start) || entry.Timestamp.After(end) || entry.Timestamp.Equal(end) {
continue
}
extractedStream.Entries = append(extractedStream.Entries, entry)
}
extractedResp.Data.Result = append(extractedResp.Data.Result, extractedStream)
}
return &extractedResp
}
func isEmpty(lokiRes *LokiResponse) bool {
return lokiRes.Status == loghttp.QueryStatusSuccess && len(lokiRes.Data.Result) == 0
}
func emptyResponse(lokiReq *LokiRequest) *LokiResponse {
return &LokiResponse{
Status: loghttp.QueryStatusSuccess,
Statistics: stats.Result{},
Direction: lokiReq.Direction,
Limit: lokiReq.Limit,
Version: uint32(loghttp.GetVersion(lokiReq.Path)),
Data: LokiData{
ResultType: loghttp.ResultTypeStream,
Result: []logproto.Stream{},
},
}
}