Add cache usage statistics (#6317)

* Adding cache statistics

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Adding metrics to metrics.go

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Creating new stats context for use in metric queries middleware

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Clean up unnecessary log fields

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Fixing tests

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Adding stats tests

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* CHANGELOG entry

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Appeasing the linter

Documenting function

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Moving CHANGELOG entry to appropriate section

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Implementing a stats collector cache wrapper to simplify stats collection

If we keep the stats collection in pkg/storage/chunk/cache/instrumented.go, then any implementation that wraps it will cause the stats collected to be incomplete. For example: NewBackground(cacheName, cfg.Background, Instrument(cacheName, cache, reg), reg)) - the background cache requests are not collected

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>

* Fixing tests

Signed-off-by: Danny Kopping <danny.kopping@grafana.com>
pull/6330/head
Danny Kopping 3 years ago committed by GitHub
parent 65e3148bc9
commit 36e0979cf5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 1
      CHANGELOG.md
  2. 8
      pkg/logql/metrics.go
  3. 2
      pkg/logql/metrics_test.go
  4. 140
      pkg/logqlmodel/stats/context.go
  5. 68
      pkg/logqlmodel/stats/context_test.go
  6. 939
      pkg/logqlmodel/stats/stats.pb.go
  7. 28
      pkg/logqlmodel/stats/stats.proto
  8. 32
      pkg/querier/queryrange/codec_test.go
  9. 26
      pkg/querier/queryrange/prometheus_test.go
  10. 11
      pkg/querier/queryrange/queryrangebase/results_cache_test.go
  11. 3
      pkg/querier/queryrange/roundtrip.go
  12. 8
      pkg/querier/queryrange/stats.go
  13. 18
      pkg/storage/chunk/cache/cache.go
  14. 6
      pkg/storage/chunk/cache/cache_gen.go
  15. 6
      pkg/storage/chunk/cache/cache_test.go
  16. 11
      pkg/storage/chunk/cache/fifo_cache.go
  17. 4
      pkg/storage/chunk/cache/fifo_cache_test.go
  18. 23
      pkg/storage/chunk/cache/memcached.go
  19. 8
      pkg/storage/chunk/cache/memcached_test.go
  20. 6
      pkg/storage/chunk/cache/mock.go
  21. 21
      pkg/storage/chunk/cache/redis_cache.go
  22. 2
      pkg/storage/chunk/cache/redis_cache_test.go
  23. 6
      pkg/storage/chunk/cache/snappy.go
  24. 53
      pkg/storage/chunk/cache/stats.go
  25. 10
      pkg/storage/chunk/cache/tiered.go
  26. 12
      pkg/storage/chunk/fetcher/fetcher.go
  27. 3
      pkg/storage/chunk/tests/caching_fixtures_test.go
  28. 8
      pkg/storage/store.go
  29. 12
      pkg/storage/stores/series/index/caching_index_client_test.go
  30. 3
      pkg/storage/stores/series/series_store_test.go
  31. 3
      pkg/storage/util_test.go
  32. 26
      pkg/util/marshal/legacy/marshal_test.go
  33. 78
      pkg/util/marshal/marshal_test.go

@ -74,6 +74,7 @@
#### Loki
##### Enhancements
* [6317](https://github.com/grafana/loki/pull/6317/files) **dannykoping**: General: add cache usage statistics
##### Fixes
* [6152](https://github.com/grafana/loki/pull/6152) **slim-bean**: Fixes unbounded ingester memory growth when live tailing under specific circumstances.

@ -126,6 +126,14 @@ func RecordRangeAndInstantQueryMetrics(
"total_entries", stats.Summary.TotalEntriesReturned,
"queue_time", logql_stats.ConvertSecondsToNanoseconds(stats.Summary.QueueTime),
"subqueries", stats.Summary.Subqueries,
"cache_chunk_req", stats.Caches.Chunk.EntriesRequested,
"cache_chunk_hit", stats.Caches.Chunk.EntriesFound,
"cache_chunk_bytes_stored", stats.Caches.Chunk.BytesSent,
"cache_chunk_bytes_fetched", stats.Caches.Chunk.BytesReceived,
"cache_index_req", stats.Caches.Index.EntriesRequested,
"cache_index_hit", stats.Caches.Index.EntriesFound,
"cache_result_req", stats.Caches.Result.EntriesRequested,
"cache_result_hit", stats.Caches.Result.EntriesFound,
}...)
logValues = append(logValues, tagsToKeyValues(queryTags)...)

@ -84,7 +84,7 @@ func TestLogSlowQuery(t *testing.T) {
}, logqlmodel.Streams{logproto.Stream{Entries: make([]logproto.Entry, 10)}})
require.Equal(t,
fmt.Sprintf(
"level=info org_id=foo traceID=%s latency=slow query=\"{foo=\\\"bar\\\"} |= \\\"buzz\\\"\" query_type=filter range_type=range length=1h0m0s step=1m0s duration=25.25s status=200 limit=1000 returned_lines=10 throughput=100kB total_bytes=100kB total_entries=10 queue_time=2ns subqueries=0 source=logvolhist feature=beta\n",
"level=info org_id=foo traceID=%s latency=slow query=\"{foo=\\\"bar\\\"} |= \\\"buzz\\\"\" query_type=filter range_type=range length=1h0m0s step=1m0s duration=25.25s status=200 limit=1000 returned_lines=10 throughput=100kB total_bytes=100kB total_entries=10 queue_time=2ns subqueries=0 cache_chunk_req=0 cache_chunk_hit=0 cache_chunk_bytes_stored=0 cache_chunk_bytes_fetched=0 cache_index_req=0 cache_index_hit=0 cache_result_req=0 cache_result_hit=0 source=logvolhist feature=beta\n",
sp.Context().(jaeger.SpanContext).SpanID().String(),
),
buf.String())

@ -43,6 +43,7 @@ const (
type Context struct {
querier Querier
ingester Ingester
caches Caches
// store is the store statistics collected across the query path
store Store
@ -52,6 +53,15 @@ type Context struct {
mtx sync.Mutex
}
type CacheType string
const (
ChunkCache CacheType = "chunk" //nolint:staticcheck
IndexCache = "index"
ResultCache = "result"
WriteDedupeCache = "write-dedupe"
)
// NewContext creates a new statistics context
func NewContext(ctx context.Context) (*Context, context.Context) {
contextData := &Context{}
@ -79,6 +89,15 @@ func (c *Context) Ingester() Ingester {
}
}
// Caches returns the cache statistics accumulated so far.
func (c *Context) Caches() Caches {
return Caches{
Chunk: c.caches.Chunk,
Index: c.caches.Index,
Result: c.caches.Result,
}
}
// Reset clears the statistics.
func (c *Context) Reset() {
c.mtx.Lock()
@ -88,6 +107,7 @@ func (c *Context) Reset() {
c.querier.Reset()
c.ingester.Reset()
c.result.Reset()
c.caches.Reset()
}
// Result calculates the summary based on store and ingester data.
@ -99,6 +119,7 @@ func (c *Context) Result(execTime time.Duration, queueTime time.Duration, totalE
Store: c.store,
},
Ingester: c.ingester,
Caches: c.caches,
})
r.ComputeSummary(execTime, queueTime, totalEntriesReturned)
@ -168,12 +189,28 @@ func (i *Ingester) Merge(m Ingester) {
i.TotalReached += m.TotalReached
}
func (c *Caches) Merge(m Caches) {
c.Chunk.Merge(m.Chunk)
c.Index.Merge(m.Index)
c.Result.Merge(m.Result)
}
func (c *Cache) Merge(m Cache) {
c.EntriesFound += m.EntriesFound
c.EntriesRequested += m.EntriesRequested
c.EntriesStored += m.EntriesStored
c.Requests += m.Requests
c.BytesSent += m.BytesSent
c.BytesReceived += m.BytesReceived
}
// Merge merges two results of statistics.
// This will increase the total number of Subqueries.
func (r *Result) Merge(m Result) {
r.Summary.Subqueries++
r.Querier.Merge(m.Querier)
r.Ingester.Merge(m.Ingester)
r.Caches.Merge(m.Caches)
r.ComputeSummary(ConvertSecondsToNanoseconds(r.Summary.ExecTime+m.Summary.ExecTime),
ConvertSecondsToNanoseconds(r.Summary.QueueTime+m.Summary.QueueTime), int(r.Summary.TotalEntriesReturned))
}
@ -257,6 +294,85 @@ func (c *Context) AddChunksRef(i int64) {
atomic.AddInt64(&c.store.TotalChunksRef, i)
}
// AddCacheEntriesFound counts the number of cache entries requested and found
func (c *Context) AddCacheEntriesFound(t CacheType, i int) {
stats := c.getCacheStatsByType(t)
if stats == nil {
return
}
atomic.AddInt32(&stats.EntriesFound, int32(i))
}
// AddCacheEntriesRequested counts the number of keys requested from the cache
func (c *Context) AddCacheEntriesRequested(t CacheType, i int) {
stats := c.getCacheStatsByType(t)
if stats == nil {
return
}
atomic.AddInt32(&stats.EntriesRequested, int32(i))
}
// AddCacheEntriesStored counts the number of keys *attempted* to be stored in the cache
// It should be noted that if a background writeback (https://grafana.com/docs/loki/latest/configuration/#cache_config)
// is configured we cannot know if these store attempts succeeded or not as this happens asynchronously
func (c *Context) AddCacheEntriesStored(t CacheType, i int) {
stats := c.getCacheStatsByType(t)
if stats == nil {
return
}
atomic.AddInt32(&stats.EntriesStored, int32(i))
}
// AddCacheBytesRetrieved counts the amount of bytes retrieved from the cache
func (c *Context) AddCacheBytesRetrieved(t CacheType, i int) {
stats := c.getCacheStatsByType(t)
if stats == nil {
return
}
atomic.AddInt64(&stats.BytesReceived, int64(i))
}
// AddCacheBytesSent counts the amount of bytes sent to the cache
// It should be noted that if a background writeback (https://grafana.com/docs/loki/latest/configuration/#cache_config)
// is configured we cannot know if these bytes actually got stored or not as this happens asynchronously
func (c *Context) AddCacheBytesSent(t CacheType, i int) {
stats := c.getCacheStatsByType(t)
if stats == nil {
return
}
atomic.AddInt64(&stats.BytesSent, int64(i))
}
// AddCacheRequest counts the number of fetch/store requests to the cache
func (c *Context) AddCacheRequest(t CacheType, i int) {
stats := c.getCacheStatsByType(t)
if stats == nil {
return
}
atomic.AddInt32(&stats.Requests, int32(i))
}
func (c *Context) getCacheStatsByType(t CacheType) *Cache {
var stats *Cache
switch t {
case ChunkCache:
stats = &c.caches.Chunk
case IndexCache:
stats = &c.caches.Index
case ResultCache:
stats = &c.caches.Result
default:
return nil
}
return stats
}
// Log logs a query statistics result.
func (r Result) Log(log log.Logger) {
_ = log.Log(
@ -284,6 +400,7 @@ func (r Result) Log(log log.Logger) {
"Querier.CompressedBytes", humanize.Bytes(uint64(r.Querier.Store.Chunk.CompressedBytes)),
"Querier.TotalDuplicates", r.Querier.Store.Chunk.TotalDuplicates,
)
r.Caches.Log(log)
r.Summary.Log(log)
}
@ -297,3 +414,26 @@ func (s Summary) Log(log log.Logger) {
"Summary.QueueTime", ConvertSecondsToNanoseconds(s.QueueTime),
)
}
func (c Caches) Log(log log.Logger) {
_ = log.Log(
"Cache.Chunk.Requests", c.Chunk.Requests,
"Cache.Chunk.EntriesRequested", c.Chunk.EntriesRequested,
"Cache.Chunk.EntriesFound", c.Chunk.EntriesFound,
"Cache.Chunk.EntriesStored", c.Chunk.EntriesStored,
"Cache.Chunk.BytesSent", humanize.Bytes(uint64(c.Chunk.BytesSent)),
"Cache.Chunk.BytesReceived", humanize.Bytes(uint64(c.Chunk.BytesReceived)),
"Cache.Index.Requests", c.Index.Requests,
"Cache.Index.EntriesRequested", c.Index.EntriesRequested,
"Cache.Index.EntriesFound", c.Index.EntriesFound,
"Cache.Index.EntriesStored", c.Index.EntriesStored,
"Cache.Index.BytesSent", humanize.Bytes(uint64(c.Index.BytesSent)),
"Cache.Index.BytesReceived", humanize.Bytes(uint64(c.Index.BytesReceived)),
"Cache.Result.Requests", c.Result.Requests,
"Cache.Result.EntriesRequested", c.Result.EntriesRequested,
"Cache.Result.EntriesFound", c.Result.EntriesFound,
"Cache.Result.EntriesStored", c.Result.EntriesStored,
"Cache.Result.BytesSent", humanize.Bytes(uint64(c.Result.BytesSent)),
"Cache.Result.BytesReceived", humanize.Bytes(uint64(c.Result.BytesReceived)),
)
}

@ -22,6 +22,9 @@ func TestResult(t *testing.T) {
stats.AddChunksRef(50)
stats.AddChunksDownloaded(60)
stats.AddChunksDownloadTime(time.Second)
stats.AddCacheRequest(ChunkCache, 3)
stats.AddCacheRequest(IndexCache, 4)
stats.AddCacheRequest(ResultCache, 1)
fakeIngesterQuery(ctx)
fakeIngesterQuery(ctx)
@ -60,6 +63,17 @@ func TestResult(t *testing.T) {
},
},
},
Caches: Caches{
Chunk: Cache{
Requests: 3,
},
Index: Cache{
Requests: 4,
},
Result: Cache{
Requests: 1,
},
},
Summary: Summary{
ExecTime: 2 * time.Second.Seconds(),
QueueTime: 2 * time.Nanosecond.Seconds(),
@ -182,6 +196,20 @@ func TestResult_Merge(t *testing.T) {
},
},
},
Caches: Caches{
Chunk: Cache{
Requests: 5,
BytesReceived: 1024,
BytesSent: 512,
},
Index: Cache{
EntriesRequested: 22,
EntriesFound: 2,
},
Result: Cache{
EntriesStored: 3,
},
},
Summary: Summary{
ExecTime: 2 * time.Second.Seconds(),
QueueTime: 2 * time.Nanosecond.Seconds(),
@ -230,6 +258,20 @@ func TestResult_Merge(t *testing.T) {
},
},
},
Caches: Caches{
Chunk: Cache{
Requests: 2 * 5,
BytesReceived: 2 * 1024,
BytesSent: 2 * 512,
},
Index: Cache{
EntriesRequested: 2 * 22,
EntriesFound: 2 * 2,
},
Result: Cache{
EntriesStored: 2 * 3,
},
},
Summary: Summary{
ExecTime: 2 * 2 * time.Second.Seconds(),
QueueTime: 2 * 2 * time.Nanosecond.Seconds(),
@ -273,3 +315,29 @@ func TestIngester(t *testing.T) {
},
}, statsCtx.Ingester())
}
func TestCaches(t *testing.T) {
statsCtx, _ := NewContext(context.Background())
statsCtx.AddCacheRequest(ChunkCache, 5)
statsCtx.AddCacheEntriesStored(ResultCache, 3)
statsCtx.AddCacheEntriesRequested(IndexCache, 22)
statsCtx.AddCacheBytesRetrieved(ChunkCache, 1024)
statsCtx.AddCacheBytesSent(ChunkCache, 512)
statsCtx.AddCacheEntriesFound(IndexCache, 2)
require.Equal(t, Caches{
Chunk: Cache{
Requests: 5,
BytesReceived: 1024,
BytesSent: 512,
},
Index: Cache{
EntriesRequested: 22,
EntriesFound: 2,
},
Result: Cache{
EntriesStored: 3,
},
}, statsCtx.Caches())
}

File diff suppressed because it is too large Load Diff

@ -22,6 +22,25 @@ message Result {
(gogoproto.nullable) = false,
(gogoproto.jsontag) = "ingester"
];
Caches caches = 4 [
(gogoproto.nullable) = false,
(gogoproto.jsontag) = "cache"
];
}
message Caches {
Cache chunk = 1 [
(gogoproto.nullable) = false,
(gogoproto.jsontag) = "chunk"
];
Cache index = 2 [
(gogoproto.nullable) = false,
(gogoproto.jsontag) = "index"
];
Cache result = 3 [
(gogoproto.nullable) = false,
(gogoproto.jsontag) = "result"
];
}
// Summary is the summary of a query statistics.
@ -99,3 +118,12 @@ message Chunk {
// Total duplicates found while processing.
int64 totalDuplicates = 9 [(gogoproto.jsontag) = "totalDuplicates"];
}
message Cache {
int32 entriesFound = 1 [(gogoproto.jsontag) = "entriesFound"];
int32 entriesRequested = 2 [(gogoproto.jsontag) = "entriesRequested"];
int32 entriesStored = 3 [(gogoproto.jsontag) = "entriesStored"];
int64 bytesReceived = 4 [(gogoproto.jsontag) = "bytesReceived"];
int64 bytesSent = 5 [(gogoproto.jsontag) = "bytesSent"];
int32 requests = 6 [(gogoproto.jsontag) = "requests"];
}

@ -927,6 +927,32 @@ var (
"totalChunksDownloaded": 18
}
},
"cache": {
"chunk": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"index": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"result": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
}
},
"summary": {
"bytesProcessedPerSecond": 20,
"execTime": 22,
@ -1121,6 +1147,12 @@ var (
TotalLinesSent: 9,
TotalReached: 10,
},
Caches: stats.Caches{
Chunk: stats.Cache{},
Index: stats.Cache{},
Result: stats.Cache{},
},
}
)

@ -47,6 +47,32 @@ var emptyStats = `"stats": {
}
}
},
"cache": {
"chunk": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"index": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"result": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
}
},
"summary": {
"bytesProcessedPerSecond": 0,
"execTime": 0,

@ -16,6 +16,7 @@ import (
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/storage/chunk/cache"
)
@ -752,7 +753,7 @@ func TestResultsCache(t *testing.T) {
Cache: cache.NewMockCache(),
},
}
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger())
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger(), stats.ResultCache)
require.NoError(t, err)
rcm, err := NewResultsCacheMiddleware(
log.NewNopLogger(),
@ -794,7 +795,7 @@ func TestResultsCacheRecent(t *testing.T) {
var cfg ResultsCacheConfig
flagext.DefaultValues(&cfg)
cfg.CacheConfig.Cache = cache.NewMockCache()
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger())
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger(), stats.ResultCache)
require.NoError(t, err)
rcm, err := NewResultsCacheMiddleware(
log.NewNopLogger(),
@ -857,7 +858,7 @@ func TestResultsCacheMaxFreshness(t *testing.T) {
var cfg ResultsCacheConfig
flagext.DefaultValues(&cfg)
cfg.CacheConfig.Cache = cache.NewMockCache()
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger())
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger(), stats.ResultCache)
require.NoError(t, err)
fakeLimits := tc.fakeLimits
rcm, err := NewResultsCacheMiddleware(
@ -897,7 +898,7 @@ func Test_resultsCache_MissingData(t *testing.T) {
Cache: cache.NewMockCache(),
},
}
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger())
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger(), stats.ResultCache)
require.NoError(t, err)
rm, err := NewResultsCacheMiddleware(
log.NewNopLogger(),
@ -1008,7 +1009,7 @@ func TestResultsCacheShouldCacheFunc(t *testing.T) {
var cfg ResultsCacheConfig
flagext.DefaultValues(&cfg)
cfg.CacheConfig.Cache = cache.NewMockCache()
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger())
c, err := cache.New(cfg.CacheConfig, nil, log.NewNopLogger(), stats.ResultCache)
require.NoError(t, err)
rcm, err := NewResultsCacheMiddleware(
log.NewNopLogger(),

@ -15,6 +15,7 @@ import (
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logql/syntax"
"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/storage/config"
@ -52,7 +53,7 @@ func NewTripperware(
err error
)
if cfg.CacheResults {
c, err = cache.New(cfg.CacheConfig, registerer, log)
c, err = cache.New(cfg.CacheConfig, registerer, log, stats.ResultCache)
if err != nil {
return nil, nil, err
}

@ -110,8 +110,11 @@ func StatsCollectorMiddleware() queryrangebase.Middleware {
logger := spanlogger.FromContext(ctx)
start := time.Now()
// start a new statistics context to be used by middleware, which we will merge with the response's statistics
st, statsCtx := stats.NewContext(ctx)
// execute the request
resp, err := next.Do(ctx, req)
resp, err := next.Do(statsCtx, req)
// collect stats and status
var statistics *stats.Result
@ -145,6 +148,9 @@ func StatsCollectorMiddleware() queryrangebase.Middleware {
}
if statistics != nil {
// merge the response's statistics with the stats collected by the middleware
statistics.Merge(st.Result(time.Since(start), 0, totalEntries))
// Re-calculate the summary: the queueTime result is already merged so should not be updated
// Log and record metrics for the current query
statistics.ComputeSummary(time.Since(start), 0, totalEntries)

@ -9,6 +9,8 @@ import (
"github.com/go-kit/log"
"github.com/prometheus/client_golang/prometheus"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
// Cache byte arrays by key.
@ -22,6 +24,8 @@ type Cache interface {
Store(ctx context.Context, key []string, buf [][]byte) error
Fetch(ctx context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error)
Stop()
// GetCacheType returns a string indicating the cache "type" for the purpose of grouping cache usage statistics
GetCacheType() stats.CacheType
}
// Config for building Caches.
@ -83,7 +87,7 @@ func IsRedisSet(cfg Config) bool {
}
// New creates a new Cache using Config.
func New(cfg Config, reg prometheus.Registerer, logger log.Logger) (Cache, error) {
func New(cfg Config, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) (Cache, error) {
if cfg.Cache != nil {
return cfg.Cache, nil
}
@ -95,8 +99,8 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger) (Cache, error
cfg.Fifocache.TTL = cfg.DefaultValidity
}
if cache := NewFifoCache(cfg.Prefix+"fifocache", cfg.Fifocache, reg, logger); cache != nil {
caches = append(caches, Instrument(cfg.Prefix+"fifocache", cache, reg))
if cache := NewFifoCache(cfg.Prefix+"fifocache", cfg.Fifocache, reg, logger, cacheType); cache != nil {
caches = append(caches, CollectStats(Instrument(cfg.Prefix+"fifocache", cache, reg)))
}
}
@ -110,10 +114,10 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger) (Cache, error
}
client := NewMemcachedClient(cfg.MemcacheClient, cfg.Prefix, reg, logger)
cache := NewMemcached(cfg.Memcache, client, cfg.Prefix, reg, logger)
cache := NewMemcached(cfg.Memcache, client, cfg.Prefix, reg, logger, cacheType)
cacheName := cfg.Prefix + "memcache"
caches = append(caches, NewBackground(cacheName, cfg.Background, Instrument(cacheName, cache, reg), reg))
caches = append(caches, CollectStats(NewBackground(cacheName, cfg.Background, Instrument(cacheName, cache, reg), reg)))
}
if IsRedisSet(cfg) {
@ -125,8 +129,8 @@ func New(cfg Config, reg prometheus.Registerer, logger log.Logger) (Cache, error
if err != nil {
return nil, fmt.Errorf("redis client setup failed: %w", err)
}
cache := NewRedisCache(cacheName, client, logger)
caches = append(caches, NewBackground(cacheName, cfg.Background, Instrument(cacheName, cache, reg), reg))
cache := NewRedisCache(cacheName, client, logger, cacheType)
caches = append(caches, CollectStats(NewBackground(cacheName, cfg.Background, Instrument(cacheName, cache, reg), reg)))
}
cache := NewTiered(caches)

@ -2,6 +2,8 @@ package cache
import (
"context"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
type contextKey int
@ -44,6 +46,10 @@ func (c GenNumMiddleware) Stop() {
c.downstreamCache.Stop()
}
func (c GenNumMiddleware) GetCacheType() stats.CacheType {
return c.downstreamCache.GetCacheType()
}
// InjectCacheGenNumber returns a derived context containing the cache gen.
func InjectCacheGenNumber(ctx context.Context, cacheGen string) context.Context {
return context.WithValue(ctx, interface{}(cacheGenContextKey), cacheGen)

@ -188,7 +188,7 @@ func testCache(t *testing.T, cache cache.Cache) {
func TestMemcache(t *testing.T) {
t.Run("Unbatched", func(t *testing.T) {
cache := cache.NewMemcached(cache.MemcachedConfig{}, newMockMemcache(),
"test", nil, log.NewNopLogger())
"test", nil, log.NewNopLogger(), "test")
testCache(t, cache)
})
@ -196,14 +196,14 @@ func TestMemcache(t *testing.T) {
cache := cache.NewMemcached(cache.MemcachedConfig{
BatchSize: 10,
Parallelism: 3,
}, newMockMemcache(), "test", nil, log.NewNopLogger())
}, newMockMemcache(), "test", nil, log.NewNopLogger(), "test")
testCache(t, cache)
})
}
func TestFifoCache(t *testing.T) {
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 1e3, TTL: 1 * time.Hour},
nil, log.NewNopLogger())
nil, log.NewNopLogger(), "test")
testCache(t, cache)
}

@ -17,6 +17,7 @@ import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/grafana/loki/pkg/logqlmodel/stats"
util_log "github.com/grafana/loki/pkg/util/log"
)
@ -71,6 +72,8 @@ func parsebytes(s string) (uint64, error) {
// FifoCache is a simple string -> interface{} cache which uses a fifo slide to
// manage evictions. O(1) inserts and updates, O(1) gets.
type FifoCache struct {
cacheType stats.CacheType
lock sync.RWMutex
maxSizeItems int
maxSizeBytes uint64
@ -98,7 +101,7 @@ type cacheEntry struct {
}
// NewFifoCache returns a new initialised FifoCache of size.
func NewFifoCache(name string, cfg FifoCacheConfig, reg prometheus.Registerer, logger log.Logger) *FifoCache {
func NewFifoCache(name string, cfg FifoCacheConfig, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) *FifoCache {
util_log.WarnExperimentalUse(fmt.Sprintf("In-memory (FIFO) cache - %s", name), logger)
if cfg.DeprecatedSize > 0 {
@ -127,6 +130,8 @@ func NewFifoCache(name string, cfg FifoCacheConfig, reg prometheus.Registerer, l
}
cache := &FifoCache{
cacheType: cacheType,
maxSizeItems: cfg.MaxSizeItems,
maxSizeBytes: maxSizeBytes,
entries: make(map[string]*list.Element),
@ -283,6 +288,10 @@ func (c *FifoCache) Stop() {
c.memoryBytes.Set(float64(0))
}
func (c *FifoCache) GetCacheType() stats.CacheType {
return c.cacheType
}
func (c *FifoCache) put(key string, value []byte) {
// See if we already have the item in the cache.
element, ok := c.entries[key]

@ -38,7 +38,7 @@ func TestFifoCacheEviction(t *testing.T) {
}
for _, test := range tests {
c := NewFifoCache(test.name, test.cfg, nil, log.NewNopLogger())
c := NewFifoCache(test.name, test.cfg, nil, log.NewNopLogger(), "test")
ctx := context.Background()
// Check put / get works
@ -193,7 +193,7 @@ func TestFifoCacheExpiry(t *testing.T) {
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
c := NewFifoCache(test.name, test.cfg, nil, log.NewNopLogger())
c := NewFifoCache(test.name, test.cfg, nil, log.NewNopLogger(), "test")
ctx := context.Background()
err := c.Store(ctx, []string{key1, key2, key3, key4}, [][]byte{data1, data2, data3, data4})

@ -15,6 +15,7 @@ import (
"github.com/prometheus/client_golang/prometheus/promauto"
instr "github.com/weaveworks/common/instrument"
"github.com/grafana/loki/pkg/logqlmodel/stats"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/math"
)
@ -36,9 +37,10 @@ func (cfg *MemcachedConfig) RegisterFlagsWithPrefix(prefix, description string,
// Memcached type caches chunks in memcached
type Memcached struct {
cfg MemcachedConfig
memcache MemcachedClient
name string
cfg MemcachedConfig
memcache MemcachedClient
name string
cacheType stats.CacheType
requestDuration *instr.HistogramCollector
@ -49,12 +51,13 @@ type Memcached struct {
}
// NewMemcached makes a new Memcached.
func NewMemcached(cfg MemcachedConfig, client MemcachedClient, name string, reg prometheus.Registerer, logger log.Logger) *Memcached {
func NewMemcached(cfg MemcachedConfig, client MemcachedClient, name string, reg prometheus.Registerer, logger log.Logger, cacheType stats.CacheType) *Memcached {
c := &Memcached{
cfg: cfg,
memcache: client,
name: name,
logger: logger,
cfg: cfg,
memcache: client,
name: name,
logger: logger,
cacheType: cacheType,
requestDuration: instr.NewHistogramCollector(
promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{
Namespace: "loki",
@ -234,6 +237,10 @@ func (c *Memcached) Stop() {
c.wg.Wait()
}
func (c *Memcached) GetCacheType() stats.CacheType {
return c.cacheType
}
// HashKey hashes key into something you can store in memcached.
func HashKey(key string) string {
hasher := fnv.New64a()

@ -18,7 +18,7 @@ func TestMemcached(t *testing.T) {
t.Run("unbatched", func(t *testing.T) {
client := newMockMemcache()
memcache := cache.NewMemcached(cache.MemcachedConfig{}, client,
"test", nil, log.NewNopLogger())
"test", nil, log.NewNopLogger(), "test")
testMemcache(t, memcache)
})
@ -28,7 +28,7 @@ func TestMemcached(t *testing.T) {
memcache := cache.NewMemcached(cache.MemcachedConfig{
BatchSize: 10,
Parallelism: 5,
}, client, "test", nil, log.NewNopLogger())
}, client, "test", nil, log.NewNopLogger(), "test")
testMemcache(t, memcache)
})
@ -95,7 +95,7 @@ func TestMemcacheFailure(t *testing.T) {
t.Run("unbatched", func(t *testing.T) {
client := newMockMemcacheFailing()
memcache := cache.NewMemcached(cache.MemcachedConfig{}, client,
"test", nil, log.NewNopLogger())
"test", nil, log.NewNopLogger(), "test")
testMemcacheFailing(t, memcache)
})
@ -105,7 +105,7 @@ func TestMemcacheFailure(t *testing.T) {
memcache := cache.NewMemcached(cache.MemcachedConfig{
BatchSize: 10,
Parallelism: 5,
}, client, "test", nil, log.NewNopLogger())
}, client, "test", nil, log.NewNopLogger(), "test")
testMemcacheFailing(t, memcache)
})

@ -3,6 +3,8 @@ package cache
import (
"context"
"sync"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
type mockCache struct {
@ -37,6 +39,10 @@ func (m *mockCache) Fetch(ctx context.Context, keys []string) (found []string, b
func (m *mockCache) Stop() {
}
func (m *mockCache) GetCacheType() stats.CacheType {
return "mock"
}
// NewMockCache makes a new MockCache.
func NewMockCache() Cache {
return &mockCache{

@ -7,23 +7,26 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/loki/pkg/logqlmodel/stats"
util_log "github.com/grafana/loki/pkg/util/log"
)
// RedisCache type caches chunks in redis
type RedisCache struct {
name string
redis *RedisClient
logger log.Logger
name string
cacheType stats.CacheType
redis *RedisClient
logger log.Logger
}
// NewRedisCache creates a new RedisCache
func NewRedisCache(name string, redisClient *RedisClient, logger log.Logger) *RedisCache {
func NewRedisCache(name string, redisClient *RedisClient, logger log.Logger, cacheType stats.CacheType) *RedisCache {
util_log.WarnExperimentalUse(fmt.Sprintf("Redis cache - %s", name), logger)
cache := &RedisCache{
name: name,
redis: redisClient,
logger: logger,
name: name,
redis: redisClient,
logger: logger,
cacheType: cacheType,
}
if err := cache.redis.Ping(context.Background()); err != nil {
level.Error(logger).Log("msg", "error connecting to redis", "name", name, "err", err)
@ -64,3 +67,7 @@ func (c *RedisCache) Store(ctx context.Context, keys []string, bufs [][]byte) er
func (c *RedisCache) Stop() {
_ = c.redis.Close()
}
func (c *RedisCache) GetCacheType() stats.CacheType {
return c.cacheType
}

@ -64,5 +64,5 @@ func mockRedisCache() (*RedisCache, error) {
Addrs: []string{redisServer.Addr()},
}),
}
return NewRedisCache("mock", redisClient, log.NewNopLogger()), nil
return NewRedisCache("mock", redisClient, log.NewNopLogger(), "test"), nil
}

@ -6,6 +6,8 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/golang/snappy"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
type snappyCache struct {
@ -47,3 +49,7 @@ func (s *snappyCache) Fetch(ctx context.Context, keys []string) ([]string, [][]b
func (s *snappyCache) Stop() {
s.next.Stop()
}
func (c *snappyCache) GetCacheType() stats.CacheType {
return c.next.GetCacheType()
}

@ -0,0 +1,53 @@
package cache
import (
"context"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
type statsCollector struct {
Cache
}
// CollectStats returns a new Cache that keeps various statistics on cache usage.
func CollectStats(cache Cache) Cache {
return &statsCollector{
Cache: cache,
}
}
func (s statsCollector) Store(ctx context.Context, keys []string, bufs [][]byte) error {
st := stats.FromContext(ctx)
st.AddCacheRequest(s.Cache.GetCacheType(), 1)
// we blindly count the number of keys to be stored since we can't know if these will actually be written back to
// the cache successfully if cache.backgroundCache is in use
st.AddCacheEntriesStored(s.Cache.GetCacheType(), len(keys))
return s.Cache.Store(ctx, keys, bufs)
}
func (s statsCollector) Fetch(ctx context.Context, keys []string) (found []string, bufs [][]byte, missing []string, err error) {
st := stats.FromContext(ctx)
st.AddCacheRequest(s.Cache.GetCacheType(), 1)
found, bufs, missing, err = s.Cache.Fetch(ctx, keys)
st.AddCacheEntriesFound(s.Cache.GetCacheType(), len(found))
st.AddCacheEntriesRequested(s.Cache.GetCacheType(), len(keys))
for j := range bufs {
st.AddCacheBytesRetrieved(s.Cache.GetCacheType(), len(bufs[j]))
}
return found, bufs, missing, err
}
func (s statsCollector) Stop() {
s.Cache.Stop()
}
func (s statsCollector) GetCacheType() stats.CacheType {
return s.Cache.GetCacheType()
}

@ -1,6 +1,10 @@
package cache
import "context"
import (
"context"
"github.com/grafana/loki/pkg/logqlmodel/stats"
)
type tiered []Cache
@ -78,3 +82,7 @@ func (t tiered) Stop() {
c.Stop()
}
}
func (t tiered) GetCacheType() stats.CacheType {
return "tiered"
}

@ -10,6 +10,7 @@ import (
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/prometheus/promql"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/storage/chunk/client"
@ -182,6 +183,17 @@ func (c *Fetcher) FetchChunks(ctx context.Context, chunks []chunk.Chunk, keys []
fromStorage, err = c.storage.GetChunks(ctx, missing)
}
// normally these stats would be collected by the cache.statsCollector wrapper, but chunks are written back
// to the cache asynchronously in the background and we lose the context
var bytes int
for _, c := range fromStorage {
bytes += c.Size()
}
st := stats.FromContext(ctx)
st.AddCacheEntriesStored(stats.ChunkCache, len(fromStorage))
st.AddCacheBytesSent(stats.ChunkCache, bytes)
// Always cache any chunks we did get
if cacheErr := c.writeBackCacheAsync(fromStorage); cacheErr != nil {
if cacheErr == errAsyncBufferFull {

@ -8,6 +8,7 @@ import (
"github.com/grafana/dskit/flagext"
"github.com/prometheus/client_golang/prometheus"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/chunk/client/gcp"
@ -33,7 +34,7 @@ func (f fixture) Clients() (index.Client, client.Client, index.TableClient, conf
indexClient = index.NewCachingIndexClient(indexClient, cache.NewFifoCache("index-fifo", cache.FifoCacheConfig{
MaxSizeItems: 500,
TTL: 5 * time.Minute,
}, reg, logger), 5*time.Minute, limits, logger, false)
}, reg, logger, stats.ChunkCache), 5*time.Minute, limits, logger, false)
return indexClient, chunkClient, tableClient, schemaConfig, closer, err
}

@ -88,19 +88,19 @@ func NewStore(cfg Config, storeCfg config.ChunkStoreConfig, schemaCfg config.Sch
}
}
indexReadCache, err := cache.New(cfg.IndexQueriesCacheConfig, registerer, logger)
indexReadCache, err := cache.New(cfg.IndexQueriesCacheConfig, registerer, logger, stats.IndexCache)
if err != nil {
return nil, err
}
writeDedupeCache, err := cache.New(storeCfg.WriteDedupeCacheConfig, registerer, logger)
writeDedupeCache, err := cache.New(storeCfg.WriteDedupeCacheConfig, registerer, logger, stats.WriteDedupeCache)
if err != nil {
return nil, err
}
chunkCacheCfg := storeCfg.ChunkCacheConfig
chunkCacheCfg.Prefix = "chunks"
chunksCache, err := cache.New(chunkCacheCfg, registerer, logger)
chunksCache, err := cache.New(chunkCacheCfg, registerer, logger, stats.ChunkCache)
if err != nil {
return nil, err
}
@ -154,7 +154,7 @@ func (s *store) init() error {
if err != nil {
return err
}
f, err := fetcher.New(s.chunksCache, s.storeCfg.ChunkCacheStubs(), s.schemaCfg, chunkClient, s.storeCfg.ChunkCacheConfig.AsyncCacheWriteBackConcurrency, s.storeCfg.ChunkCacheConfig.AsyncCacheWriteBackBufferSize)
f, err := fetcher.New(cache.CollectStats(s.chunksCache), s.storeCfg.ChunkCacheStubs(), s.schemaCfg, chunkClient, s.storeCfg.ChunkCacheConfig.AsyncCacheWriteBackConcurrency, s.storeCfg.ChunkCacheConfig.AsyncCacheWriteBackBufferSize)
if err != nil {
return err
}

@ -49,7 +49,7 @@ func TestCachingStorageClientBasic(t *testing.T) {
limits, err := defaultLimits()
require.NoError(t, err)
logger := log.NewNopLogger()
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger)
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger, "test")
client := index.NewCachingIndexClient(store, cache, 1*time.Second, limits, logger, false)
queries := []index.Query{{
TableName: "table",
@ -81,7 +81,7 @@ func TestTempCachingStorageClient(t *testing.T) {
limits, err := defaultLimits()
require.NoError(t, err)
logger := log.NewNopLogger()
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger)
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger, "test")
client := index.NewCachingIndexClient(store, cache, 100*time.Millisecond, limits, logger, false)
queries := []index.Query{
{TableName: "table", HashValue: "foo"},
@ -140,7 +140,7 @@ func TestPermCachingStorageClient(t *testing.T) {
limits, err := defaultLimits()
require.NoError(t, err)
logger := log.NewNopLogger()
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger)
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger, "test")
client := index.NewCachingIndexClient(store, cache, 100*time.Millisecond, limits, logger, false)
queries := []index.Query{
{TableName: "table", HashValue: "foo", Immutable: true},
@ -196,7 +196,7 @@ func TestCachingStorageClientEmptyResponse(t *testing.T) {
limits, err := defaultLimits()
require.NoError(t, err)
logger := log.NewNopLogger()
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger)
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger, "test")
client := index.NewCachingIndexClient(store, cache, 1*time.Second, limits, logger, false)
queries := []index.Query{{TableName: "table", HashValue: "foo"}}
err = client.QueryPages(ctx, queries, func(query index.Query, batch index.ReadBatchResult) bool {
@ -235,7 +235,7 @@ func TestCachingStorageClientCollision(t *testing.T) {
limits, err := defaultLimits()
require.NoError(t, err)
logger := log.NewNopLogger()
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger)
cache := cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger, "test")
client := index.NewCachingIndexClient(store, cache, 1*time.Second, limits, logger, false)
queries := []index.Query{
{TableName: "table", HashValue: "foo", RangeValuePrefix: []byte("bar")},
@ -415,7 +415,7 @@ func TestCachingStorageClientStoreQueries(t *testing.T) {
require.NoError(t, err)
logger := log.NewNopLogger()
cache := &mockCache{
Cache: cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger),
Cache: cache.NewFifoCache("test", cache.FifoCacheConfig{MaxSizeItems: 10, TTL: 10 * time.Second}, nil, logger, "test"),
}
client := index.NewCachingIndexClient(store, cache, 1*time.Second, limits, logger, disableBroadQueries)
var callbackQueries []index.Query

@ -19,6 +19,7 @@ import (
"github.com/weaveworks/common/user"
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/cache"
@ -54,7 +55,7 @@ var (
flagext.DefaultValues(&storeCfg)
storeCfg.WriteDedupeCacheConfig.Cache = cache.NewFifoCache("test", cache.FifoCacheConfig{
MaxSizeItems: 500,
}, prometheus.NewRegistry(), log.NewNopLogger())
}, prometheus.NewRegistry(), log.NewNopLogger(), stats.ChunkCache)
return storeCfg
},
},

@ -15,6 +15,7 @@ import (
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/cache"
@ -241,7 +242,7 @@ func (m *mockChunkStore) GetChunkRefs(ctx context.Context, userID string, from,
refs = append(refs, r)
}
cache, err := cache.New(cache.Config{Prefix: "chunks"}, nil, util_log.Logger)
cache, err := cache.New(cache.Config{Prefix: "chunks"}, nil, util_log.Logger, stats.ChunkCache)
if err != nil {
panic(err)
}

@ -78,6 +78,32 @@ var queryTests = []struct {
}
}
},
"cache": {
"chunk": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"index": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"result": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
}
},
"summary": {
"bytesProcessedPerSecond": 0,
"execTime": 0,

@ -84,6 +84,32 @@ var queryTests = []struct {
}
}
},
"cache": {
"chunk": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"index": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"result": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
}
},
"summary": {
"bytesProcessedPerSecond": 0,
"execTime": 0,
@ -194,6 +220,32 @@ var queryTests = []struct {
}
}
},
"cache": {
"chunk": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"index": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"result": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
}
},
"summary": {
"bytesProcessedPerSecond": 0,
"execTime": 0,
@ -321,6 +373,32 @@ var queryTests = []struct {
}
}
},
"cache": {
"chunk": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"index": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
},
"result": {
"entriesFound": 0,
"entriesRequested": 0,
"entriesStored": 0,
"bytesReceived": 0,
"bytesSent": 0,
"requests": 0
}
},
"summary": {
"bytesProcessedPerSecond": 0,
"execTime": 0,

Loading…
Cancel
Save