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

367 lines
12 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/grafana/dskit/httpgrpc"
"github.com/grafana/dskit/tenant"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"golang.org/x/sync/errgroup"
"github.com/grafana/loki/v3/pkg/loghttp"
"github.com/grafana/loki/v3/pkg/logproto"
"github.com/grafana/loki/v3/pkg/logqlmodel/stats"
"github.com/grafana/loki/v3/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/v3/pkg/storage/chunk/cache"
"github.com/grafana/loki/v3/pkg/util/constants"
"github.com/grafana/loki/v3/pkg/util/httpreq"
"github.com/grafana/loki/v3/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: constants.Loki,
Name: "query_frontend_log_result_cache_hit_total",
}),
CacheMiss: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Namespace: constants.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) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "logResultCache.Do")
defer sp.Finish()
tenantIDs, err := tenant.TenantIDs(ctx)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, "%s", err.Error())
}
Add config to avoid caching stats for recent data (#9537) **What this PR does / why we need it**: When we query the stats for recent data, we query both the ingesters and the index gateways for the stats. https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/async_store.go#L112-L114 https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/async_store.go#L126-L127 Then we merge all the responses, which means summing up all the stats https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/async_store.go#L157-L158 https://github.com/grafana/loki/blob/ebdb2b18007e024d56105afc5230383165ca1650/pkg/storage/stores/index/stats/stats.go#L23-L26 Because we have a replication factor of 3, this means that we will get the stats from the ingesters repeated up to 3 times, hence inflating the stats. In the stats cache, we store the stats for a given matcher set for the whole day, then we extract the stats from the cache by the factor of time from the request that is stored in the cache: https://github.com/grafana/loki/blob/336283acadb34f5fda9abce4e6fcef1dca9965d8/pkg/querier/queryrange/index_stats_cache.go#L33 https://github.com/grafana/loki/blob/336283acadb34f5fda9abce4e6fcef1dca9965d8/pkg/querier/queryrange/index_stats_cache.go#L40 Inflated stats for recent data will be cached, so subsequent stats extracted from the cache will be inflated regardless of the time. This PR adds a new per-tenant limit `max_stats_cache_freshness` to not cache requests with an end time that falls within Now minus this duration. Here's a scenario illustrating this. The graphs below show the bytes stats queried in the sharding middleware. We are running a log filter query that won't match any log, every 5 seconds with a length of 3h. ![image](https://github.com/grafana/loki/assets/8354290/45c2e6e9-185c-4a18-b290-47da27fc3e39) As can be seen, after enabling the stats cache and configuring`do_not_cache_request_within` to not cache stats for requests within 30m, the bytes stats used in the sharding middleware stopped increasing. In both cases the stats cache hit ration was 100%. ![image](https://github.com/grafana/loki/assets/8354290/cd35bcb8-0c77-4693-a06b-502741fd6e23) **Special notes for your reviewer**: - Blocked by https://github.com/grafana/loki/pull/9535 - Note that this PR doesn't fix the root issue of inflated stats form the ingesters, but rather buys us some time to work on that. **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] Tests updated - [ ] `CHANGELOG.md` updated - [ ] 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)
3 years ago
if l.shouldCache != nil && !l.shouldCache(ctx, 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().UnixMilli() > 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
fix(log results cache): compose empty response based on the request (#11657) **What this PR does / why we need it**: Log results cache when handling a hit composes an empty response based on the cached request. But the limit or direction fields in the cached request need not match with the current request being served. This causes the log results cache to return a response with incorrect limit. This incorrect limit could then get applied when merging responses upstream (split by interval mw for ex.) This pr fixes this by composing the response based on the request being served. I also thought about updating the cache key to include both limit and direction to have a clear separation, but I left it as is for the following reason: if a time range contains no log lines, that result would not change irrespective of a different limit or direction **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `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/setup/upgrade/_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) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
// skip caching when limit is 0 as it would get registerted as empty result in the cache even if that time range contains log lines.
if interval == 0 || lokiReq.Limit == 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()))
if httpreq.ExtractHeader(ctx, httpreq.LokiDisablePipelineWrappersHeader) == "true" {
cacheKey = "pipeline-disabled:" + cacheKey
}
_, 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
fix(log results cache): compose empty response based on the request (#11657) **What this PR does / why we need it**: Log results cache when handling a hit composes an empty response based on the cached request. But the limit or direction fields in the cached request need not match with the current request being served. This causes the log results cache to return a response with incorrect limit. This incorrect limit could then get applied when merging responses upstream (split by interval mw for ex.) This pr fixes this by composing the response based on the request being served. I also thought about updating the cache key to include both limit and direction to have a clear separation, but I left it as is for the following reason: if a time range contains no log lines, that result would not change irrespective of a different limit or direction **Which issue(s) this PR fixes**: Fixes #<issue number> **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] Tests updated - [x] `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/setup/upgrade/_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) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
2 years ago
result := emptyResponse(lokiReq)
// 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
}
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
updateCache := false
// if the query does not overlap cached interval, do not try to fill the gap since it requires extending the queries beyond what is requested in the query.
// Extending the queries beyond what is requested could result in empty responses due to response limit set in the queries.
if !overlap(lokiReq.StartTs, lokiReq.EndTs, cachedRequest.StartTs, cachedRequest.EndTs) {
resp, err := l.next.Do(ctx, lokiReq)
if err != nil {
return nil, err
}
result = resp.(*LokiResponse)
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
// if the response is empty and the query is larger than what is cached, update the cache
if isEmpty(result) && (lokiReq.EndTs.UnixNano()-lokiReq.StartTs.UnixNano() > cachedRequest.EndTs.UnixNano()-cachedRequest.StartTs.UnixNano()) {
cachedRequest = cachedRequest.WithStartEnd(lokiReq.GetStartTs(), lokiReq.GetEndTs()).(*LokiRequest)
updateCache = true
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
}
} else {
// 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
)
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.WithStartEnd(lokiReq.GetStartTs(), cachedRequest.GetStartTs()).(*LokiRequest)
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
resp, err := l.next.Do(ctx, startRequest)
if err != nil {
return err
}
var ok bool
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.WithStartEnd(cachedRequest.GetEndTs(), lokiReq.GetEndTs()).(*LokiRequest)
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
resp, err := l.next.Do(ctx, endRequest)
if err != nil {
return err
}
var ok bool
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.WithStartEnd(startRequest.GetStartTs(), cachedRequest.GetEndTs()).(*LokiRequest)
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
updateCache = true
} else {
if startResp.Status != loghttp.QueryStatusSuccess {
return startResp, nil
}
result = mergeLokiResponse(startResp, result)
}
}
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
// 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.WithStartEnd(cachedRequest.GetStartTs(), endRequest.GetEndTs()).(*LokiRequest)
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
updateCache = true
} else {
if endResp.Status != loghttp.QueryStatusSuccess {
return endResp, nil
}
result = mergeLokiResponse(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{},
},
}
}
caching: do not try to fill the gap in log results cache when the new query interval does not overlap the cached query interval (#9757) **What this PR does / why we need it**: Currently, when we find a relevant cached negative response for a logs query, we do the following: * If the cached query completely covers the new query: * return back an empty response. * else: * fill the gaps on either/both sides of the cached query. The problem with filling the gaps is that when the cached query does not overlap at all with the new query, we have to extend the query beyond what the query requests for. However, with the logs query, we have a limit on the number of lines we can send back in the response. So, this could result in the query response having logs which were not requested by the query, which then get filtered out by the [response extractor](https://github.com/grafana/loki/blob/b78d3f05525d8bcab13e621bc2e5851aadc8fc91/pkg/querier/queryrange/log_result_cache.go#L299), unexpectedly resulting in an empty response. For example, if the query was cached for start=15, end=20 and we get a `backwards` query for start=5, end=10. To fill the gap, the query would be executed for start=5, end=15. Now, if we have logs more than the query `limit` in the range 10-15, we would filter out all the data in the response extractor and send back an empty response to the user. This PR fixes the issue by doing the following changes when handling cache hit: * If the cached query completely covers the new query: * return back an empty response[_existing_]. * else if the cached query does not overlap with the new query: * do the new query as requested. * If the new query results in an empty response and has a higher interval than the cached query: * update the cache * else: * query the data for missing intervals on both/either side[_existing_] * update the cache with extended intervals if the new queries resulted in an empty response[_existing_] **Special notes for your reviewer**: We could do further improvements in the handling of queries not overlapping with cached query by selectively extending the queries based on query direction and cached query lying before/after the new query. For example, if the new query is doing `backwards` query and the `cachedQuery.End` < `newQuery.Start`, it should be okay to extend the query and do `cachedQuery.End` to `newQuery.End` to fill the cache since query would first fill the most relevant data before hitting the limits. I did not want to complicate the fix so went without implementing this approach. We can revisit later if we feel we need to improve our caching. **Checklist** - [x] Tests updated - [x] `CHANGELOG.md` updated --------- Co-authored-by: Travis Patterson <travis.patterson@grafana.com>
3 years ago
func overlap(aFrom, aThrough, bFrom, bThrough time.Time) bool {
if aFrom.After(bThrough) || bFrom.After(aThrough) {
return false
}
return true
}