mirror of https://github.com/grafana/loki
[Frontend] Log Result Cache (#5502)
* Rough verion still require some fixes * with file saved * Finish off the code and logic Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Refactors metrics and hook the log cache middleware Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Add tests, fixes and lint Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Hash the key Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * better comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * better comment Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Update pkg/querier/queryrange/log_result_cache.go Co-authored-by: Christian Simon <simon@swine.de> * Update pkg/querier/queryrange/log_result_cache.go Co-authored-by: Christian Simon <simon@swine.de> * Fixes typo in variable namme Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Use only one ok bool Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * Review feedbacks Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> * check array boundaries Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com> Co-authored-by: Christian Simon <simon@swine.de>pull/5627/head
parent
01d63c8ae8
commit
39101308f1
@ -0,0 +1,279 @@ |
||||
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/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/tenant" |
||||
"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, 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, |
||||
metrics: metrics, |
||||
} |
||||
}) |
||||
} |
||||
|
||||
type logResultCache struct { |
||||
next queryrangebase.Handler |
||||
limits Limits |
||||
cache cache.Cache |
||||
shouldCache queryrangebase.ShouldCacheFn |
||||
|
||||
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) |
||||
} |
||||
|
||||
maxCacheFreshness := validation.MaxDurationPerTenant(tenantIDs, l.limits.MaxCacheFreshness) |
||||
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.
|
||||
cacheKey := fmt.Sprintf("log:%s:%s:%d:%d", tenant.JoinTenantIDs(tenantIDs), 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.
|
||||
if !lokiReq.GetStartTs().After(cachedRequest.GetStartTs()) && lokiReq.GetStartTs().Equal(cachedRequest.GetStartTs()) && |
||||
!lokiReq.GetEndTs().Before(cachedRequest.GetEndTs()) && lokiReq.GetEndTs().Equal(cachedRequest.GetEndTs()) { |
||||
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 |
||||
} |
||||
result = mergeLokiResponse(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 |
||||
} |
||||
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 |
||||
} |
||||
|
||||
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{}, |
||||
}, |
||||
} |
||||
} |
||||
@ -0,0 +1,452 @@ |
||||
package queryrange |
||||
|
||||
import ( |
||||
"context" |
||||
"fmt" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/go-kit/log" |
||||
"github.com/stretchr/testify/mock" |
||||
"github.com/stretchr/testify/require" |
||||
"github.com/weaveworks/common/user" |
||||
|
||||
"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" |
||||
) |
||||
|
||||
func Test_LogResultCacheSameRange(t *testing.T) { |
||||
var ( |
||||
ctx = user.InjectOrgID(context.Background(), "foo") |
||||
lrc = NewLogResultCache( |
||||
log.NewNopLogger(), |
||||
fakeLimits{ |
||||
splits: map[string]time.Duration{"foo": time.Minute}, |
||||
}, |
||||
cache.NewMockCache(), |
||||
nil, |
||||
nil, |
||||
) |
||||
) |
||||
|
||||
req := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
} |
||||
|
||||
fake := newFakeResponse([]mockResponse{ |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req, |
||||
Response: emptyResponse(req), |
||||
}, |
||||
}, |
||||
}) |
||||
|
||||
h := lrc.Wrap(fake) |
||||
|
||||
resp, err := h.Do(ctx, req) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req), resp) |
||||
resp, err = h.Do(ctx, req) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req), resp) |
||||
|
||||
fake.AssertExpectations(t) |
||||
} |
||||
|
||||
func Test_LogResultCacheSameRangeNonEmpty(t *testing.T) { |
||||
var ( |
||||
ctx = user.InjectOrgID(context.Background(), "foo") |
||||
lrc = NewLogResultCache( |
||||
log.NewNopLogger(), |
||||
fakeLimits{ |
||||
splits: map[string]time.Duration{"foo": time.Minute}, |
||||
}, |
||||
cache.NewMockCache(), |
||||
nil, |
||||
nil, |
||||
) |
||||
) |
||||
|
||||
req := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
} |
||||
|
||||
fake := newFakeResponse([]mockResponse{ |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req, |
||||
Response: nonEmptyResponse(req, 1), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req, |
||||
Response: nonEmptyResponse(req, 2), |
||||
}, |
||||
}, |
||||
}) |
||||
|
||||
h := lrc.Wrap(fake) |
||||
|
||||
resp, err := h.Do(ctx, req) |
||||
require.NoError(t, err) |
||||
require.Equal(t, nonEmptyResponse(req, 1), resp) |
||||
resp, err = h.Do(ctx, req) |
||||
require.NoError(t, err) |
||||
require.Equal(t, nonEmptyResponse(req, 2), resp) |
||||
|
||||
fake.AssertExpectations(t) |
||||
} |
||||
|
||||
func Test_LogResultCacheSmallerRange(t *testing.T) { |
||||
var ( |
||||
ctx = user.InjectOrgID(context.Background(), "foo") |
||||
lrc = NewLogResultCache( |
||||
log.NewNopLogger(), |
||||
fakeLimits{ |
||||
splits: map[string]time.Duration{"foo": time.Minute}, |
||||
}, |
||||
cache.NewMockCache(), |
||||
nil, |
||||
nil, |
||||
) |
||||
) |
||||
|
||||
req := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
} |
||||
|
||||
fake := newFakeResponse([]mockResponse{ |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req, |
||||
Response: emptyResponse(req), |
||||
}, |
||||
}, |
||||
}) |
||||
|
||||
h := lrc.Wrap(fake) |
||||
|
||||
resp, err := h.Do(ctx, req) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req), resp) |
||||
resp, err = h.Do(ctx, &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
}) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
}), resp) |
||||
|
||||
fake.AssertExpectations(t) |
||||
} |
||||
|
||||
func Test_LogResultCacheDifferentRange(t *testing.T) { |
||||
var ( |
||||
ctx = user.InjectOrgID(context.Background(), "foo") |
||||
lrc = NewLogResultCache( |
||||
log.NewNopLogger(), |
||||
fakeLimits{ |
||||
splits: map[string]time.Duration{"foo": time.Minute}, |
||||
}, |
||||
cache.NewMockCache(), |
||||
nil, |
||||
nil, |
||||
) |
||||
) |
||||
|
||||
req1 := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
} |
||||
|
||||
req2 := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
} |
||||
|
||||
fake := newFakeResponse([]mockResponse{ |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req1, |
||||
Response: emptyResponse(req1), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, |
||||
Response: emptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, |
||||
Response: emptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}), |
||||
}, |
||||
}, |
||||
}) |
||||
|
||||
h := lrc.Wrap(fake) |
||||
|
||||
resp, err := h.Do(ctx, req1) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req1), resp) |
||||
resp, err = h.Do(ctx, req2) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req2), resp) |
||||
|
||||
fake.AssertExpectations(t) |
||||
} |
||||
|
||||
func Test_LogResultCacheDifferentRangeNonEmpty(t *testing.T) { |
||||
var ( |
||||
ctx = user.InjectOrgID(context.Background(), "foo") |
||||
lrc = NewLogResultCache( |
||||
log.NewNopLogger(), |
||||
fakeLimits{ |
||||
splits: map[string]time.Duration{"foo": time.Minute}, |
||||
}, |
||||
cache.NewMockCache(), |
||||
nil, |
||||
nil, |
||||
) |
||||
) |
||||
|
||||
req1 := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
} |
||||
|
||||
req2 := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
} |
||||
|
||||
fake := newFakeResponse([]mockResponse{ |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req1, |
||||
Response: emptyResponse(req1), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, |
||||
Response: nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, 1), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, |
||||
Response: nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, 2), |
||||
}, |
||||
}, |
||||
}) |
||||
|
||||
h := lrc.Wrap(fake) |
||||
|
||||
resp, err := h.Do(ctx, req1) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req1), resp) |
||||
resp, err = h.Do(ctx, req2) |
||||
require.NoError(t, err) |
||||
require.Equal(t, mergeLokiResponse( |
||||
nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, 2), |
||||
nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, 1), |
||||
), resp) |
||||
|
||||
fake.AssertExpectations(t) |
||||
} |
||||
|
||||
func Test_LogResultCacheDifferentRangeNonEmptyAndEmpty(t *testing.T) { |
||||
var ( |
||||
ctx = user.InjectOrgID(context.Background(), "foo") |
||||
lrc = NewLogResultCache( |
||||
log.NewNopLogger(), |
||||
fakeLimits{ |
||||
splits: map[string]time.Duration{"foo": time.Minute}, |
||||
}, |
||||
cache.NewMockCache(), |
||||
nil, |
||||
nil, |
||||
) |
||||
) |
||||
|
||||
req1 := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
} |
||||
|
||||
req2 := &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
} |
||||
|
||||
fake := newFakeResponse([]mockResponse{ |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: req1, |
||||
Response: emptyResponse(req1), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, |
||||
Response: emptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}), |
||||
}, |
||||
}, |
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, |
||||
Response: nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, 2), |
||||
}, |
||||
}, |
||||
// we call it twice
|
||||
{ |
||||
RequestResponse: queryrangebase.RequestResponse{ |
||||
Request: &LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, |
||||
Response: nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, 2*time.Minute.Nanoseconds()-30*time.Second.Nanoseconds()), |
||||
EndTs: time.Unix(0, 2*time.Minute.Nanoseconds()), |
||||
}, 2), |
||||
}, |
||||
}, |
||||
}) |
||||
|
||||
h := lrc.Wrap(fake) |
||||
|
||||
resp, err := h.Do(ctx, req1) |
||||
require.NoError(t, err) |
||||
require.Equal(t, emptyResponse(req1), resp) |
||||
resp, err = h.Do(ctx, req2) |
||||
require.NoError(t, err) |
||||
require.Equal(t, mergeLokiResponse( |
||||
emptyResponse(req1), |
||||
nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, 1), |
||||
), resp) |
||||
resp, err = h.Do(ctx, req2) |
||||
require.NoError(t, err) |
||||
require.Equal(t, mergeLokiResponse( |
||||
emptyResponse(req1), |
||||
nonEmptyResponse(&LokiRequest{ |
||||
StartTs: time.Unix(0, time.Minute.Nanoseconds()), |
||||
EndTs: time.Unix(0, time.Minute.Nanoseconds()+30*time.Second.Nanoseconds()), |
||||
}, 1), |
||||
), resp) |
||||
fake.AssertExpectations(t) |
||||
} |
||||
|
||||
type fakeResponse struct { |
||||
*mock.Mock |
||||
} |
||||
|
||||
type mockResponse struct { |
||||
queryrangebase.RequestResponse |
||||
err error |
||||
} |
||||
|
||||
func newFakeResponse(responses []mockResponse) fakeResponse { |
||||
m := &mock.Mock{} |
||||
for _, r := range responses { |
||||
m.On("Do", mock.Anything, r.Request).Return(r.Response, r.err).Once() |
||||
} |
||||
return fakeResponse{ |
||||
Mock: m, |
||||
} |
||||
} |
||||
|
||||
func (f fakeResponse) Do(ctx context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { |
||||
var ( |
||||
resp queryrangebase.Response |
||||
err error |
||||
args = f.Mock.Called(ctx, r) |
||||
) |
||||
if args.Get(0) != nil { |
||||
resp = args.Get(0).(queryrangebase.Response) |
||||
} |
||||
if args.Get(1) != nil { |
||||
err = args.Get(1).(error) |
||||
} |
||||
return resp, err |
||||
} |
||||
|
||||
func nonEmptyResponse(lokiReq *LokiRequest, i int) *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{ |
||||
{ |
||||
Labels: `{foo="bar"}`, |
||||
Entries: []logproto.Entry{ |
||||
{ |
||||
Timestamp: time.Unix(1, 0), |
||||
Line: fmt.Sprintf("%d", i), |
||||
}, |
||||
}, |
||||
}, |
||||
}, |
||||
}, |
||||
} |
||||
} |
||||
@ -0,0 +1,26 @@ |
||||
package queryrange |
||||
|
||||
import ( |
||||
"github.com/prometheus/client_golang/prometheus" |
||||
|
||||
"github.com/grafana/loki/pkg/logql" |
||||
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" |
||||
) |
||||
|
||||
type Metrics struct { |
||||
*queryrangebase.InstrumentMiddlewareMetrics |
||||
*queryrangebase.RetryMiddlewareMetrics |
||||
*logql.ShardingMetrics |
||||
*SplitByMetrics |
||||
*LogResultCacheMetrics |
||||
} |
||||
|
||||
func NewMetrics(registerer prometheus.Registerer) *Metrics { |
||||
return &Metrics{ |
||||
InstrumentMiddlewareMetrics: queryrangebase.NewInstrumentMiddlewareMetrics(registerer), |
||||
RetryMiddlewareMetrics: queryrangebase.NewRetryMiddlewareMetrics(registerer), |
||||
ShardingMetrics: logql.NewShardingMetrics(registerer), |
||||
SplitByMetrics: NewSplitByMetrics(registerer), |
||||
LogResultCacheMetrics: NewLogResultCacheMetrics(registerer), |
||||
} |
||||
} |
||||
Loading…
Reference in new issue