Skip to content

Commit

Permalink
Add usage report into Loki. (#5361)
Browse files Browse the repository at this point in the history
* Adds leader election process

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* fluke

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* fixes the kv typecheck

* wire up the http client

* Hooking into loki services, hit a bug

* Add stats variable.

* re-vendor dskit and improve to never fail service

* Intrument Loki with the package

* Add changelog entry

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Fixes compactor test

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Add configuration documentation

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Update pkg/usagestats/reporter.go

Co-authored-by: Danny Kopping <dannykopping@gmail.com>

* Add boundary check

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Add log for success report.

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* lint

Signed-off-by: Cyril Tovena <cyril.tovena@gmail.com>

* Update pkg/usagestats/reporter.go

Co-authored-by: Danny Kopping <dannykopping@gmail.com>

Co-authored-by: Danny Kopping <dannykopping@gmail.com>
  • Loading branch information
cyriltovena and dannykopping authored Feb 10, 2022
1 parent 0241673 commit bbaef79
Show file tree
Hide file tree
Showing 49 changed files with 1,384 additions and 401 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
## Main

* [5361](https://github.com/grafana/loki/pull/5361) **ctovena**: Add usage report to grafana.com.
* [5289](https://github.com/grafana/loki/pull/5289) **ctovena**: Fix deduplication bug in queries when mutating labels.
* [5302](https://github.com/grafana/loki/pull/5302) **MasslessParticle** Update azure blobstore client to use new sdk.
* [5243](https://github.com/grafana/loki/pull/5290) **ssncferreira**: Update Promtail to support duration string formats.
Expand Down
10 changes: 5 additions & 5 deletions clients/pkg/promtail/targets/cloudflare/target.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,11 +109,11 @@ func (t *Target) start() {
end = maxEnd
}
start := end.Add(-time.Duration(t.config.PullRange))

requests := splitRequests(start, end, t.config.Workers)
// Use background context for workers as we don't want to cancel half way through.
// In case of errors we stop the target, each worker has it's own retry logic.
if err := concurrency.ForEach(context.Background(), splitRequests(start, end, t.config.Workers), t.config.Workers, func(ctx context.Context, job interface{}) error {
request := job.(pullRequest)
if err := concurrency.ForEachJob(context.Background(), len(requests), t.config.Workers, func(ctx context.Context, idx int) error {
request := requests[idx]
return t.pull(ctx, request.start, request.end)
}); err != nil {
level.Error(t.logger).Log("msg", "failed to pull logs", "err", err, "start", start, "end", end)
Expand Down Expand Up @@ -229,9 +229,9 @@ type pullRequest struct {
end time.Time
}

func splitRequests(start, end time.Time, workers int) []interface{} {
func splitRequests(start, end time.Time, workers int) []pullRequest {
perWorker := end.Sub(start) / time.Duration(workers)
var requests []interface{}
var requests []pullRequest
for i := 0; i < workers; i++ {
r := pullRequest{
start: start.Add(time.Duration(i) * perWorker),
Expand Down
26 changes: 13 additions & 13 deletions clients/pkg/promtail/targets/cloudflare/target_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,26 +251,26 @@ func Test_splitRequests(t *testing.T) {
tests := []struct {
start time.Time
end time.Time
want []interface{}
want []pullRequest
}{
// perfectly divisible
{
time.Unix(0, 0),
time.Unix(0, int64(time.Minute)),
[]interface{}{
pullRequest{start: time.Unix(0, 0), end: time.Unix(0, int64(time.Minute/3))},
pullRequest{start: time.Unix(0, int64(time.Minute/3)), end: time.Unix(0, int64(time.Minute*2/3))},
pullRequest{start: time.Unix(0, int64(time.Minute*2/3)), end: time.Unix(0, int64(time.Minute))},
[]pullRequest{
{start: time.Unix(0, 0), end: time.Unix(0, int64(time.Minute/3))},
{start: time.Unix(0, int64(time.Minute/3)), end: time.Unix(0, int64(time.Minute*2/3))},
{start: time.Unix(0, int64(time.Minute*2/3)), end: time.Unix(0, int64(time.Minute))},
},
},
// not divisible
{
time.Unix(0, 0),
time.Unix(0, int64(time.Minute+1)),
[]interface{}{
pullRequest{start: time.Unix(0, 0), end: time.Unix(0, int64(time.Minute/3))},
pullRequest{start: time.Unix(0, int64(time.Minute/3)), end: time.Unix(0, int64(time.Minute*2/3))},
pullRequest{start: time.Unix(0, int64(time.Minute*2/3)), end: time.Unix(0, int64(time.Minute+1))},
[]pullRequest{
{start: time.Unix(0, 0), end: time.Unix(0, int64(time.Minute/3))},
{start: time.Unix(0, int64(time.Minute/3)), end: time.Unix(0, int64(time.Minute*2/3))},
{start: time.Unix(0, int64(time.Minute*2/3)), end: time.Unix(0, int64(time.Minute+1))},
},
},
}
Expand All @@ -279,11 +279,11 @@ func Test_splitRequests(t *testing.T) {
got := splitRequests(tt.start, tt.end, 3)
if !assert.Equal(t, tt.want, got) {
for i := range got {
if !assert.Equal(t, tt.want[i].(pullRequest).start, got[i].(pullRequest).start) {
t.Logf("expected i:%d start: %d , got: %d", i, tt.want[i].(pullRequest).start.UnixNano(), got[i].(pullRequest).start.UnixNano())
if !assert.Equal(t, tt.want[i].start, got[i].start) {
t.Logf("expected i:%d start: %d , got: %d", i, tt.want[i].start.UnixNano(), got[i].start.UnixNano())
}
if !assert.Equal(t, tt.want[i].(pullRequest).end, got[i].(pullRequest).end) {
t.Logf("expected i:%d end: %d , got: %d", i, tt.want[i].(pullRequest).end.UnixNano(), got[i].(pullRequest).end.UnixNano())
if !assert.Equal(t, tt.want[i].end, got[i].end) {
t.Logf("expected i:%d end: %d , got: %d", i, tt.want[i].end.UnixNano(), got[i].end.UnixNano())
}
}
}
Expand Down
13 changes: 13 additions & 0 deletions docs/sources/configuration/_index.md
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,9 @@ Pass the `-config.expand-env` flag at the command line to enable this way of set
# If a more specific configuration is given in other sections,
# the related configuration within this section will be ignored.
[common: <common>]

# Configuration for usage report
[usage_report: <usage_report>]
```
## server
Expand Down Expand Up @@ -2496,6 +2499,16 @@ This way, one doesn't have to replicate configuration in multiple places.
[ring: <ring>]
```
## usage_report
This block allow to configure usage report of Loki to grafana.com
```yaml
# Whether or not usage report should be disabled.
# CLI flag: -usage-report.disabled
[disabled: <boolean>: default = false]
```
### storage
The common `storage` block defines a common storage to be reused by different
Expand Down
4 changes: 2 additions & 2 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ require (
github.com/google/go-cmp v0.5.6
github.com/gorilla/mux v1.8.0
github.com/gorilla/websocket v1.4.2
github.com/grafana/dskit v0.0.0-20220105080720-01ce9286d7d5
github.com/grafana/dskit v0.0.0-20220209070952-ea22a8f662d0
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645
github.com/hashicorp/consul/api v1.12.0
Expand Down Expand Up @@ -105,6 +105,7 @@ require (
github.com/Azure/azure-sdk-for-go/sdk/azidentity v0.13.0
github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v0.0.0-00010101000000-000000000000
github.com/google/renameio/v2 v2.0.0
github.com/google/uuid v1.2.0
github.com/mattn/go-ieproxy v0.0.1
github.com/xdg-go/scram v1.0.2
gopkg.in/Graylog2/go-gelf.v2 v2.0.0-20191017102106-1550ee647df0
Expand Down Expand Up @@ -185,7 +186,6 @@ require (
github.com/google/go-querystring v1.0.0 // indirect
github.com/google/gofuzz v1.1.0 // indirect
github.com/google/pprof v0.0.0-20211214055906-6f57359322fd // indirect
github.com/google/uuid v1.3.0 // indirect
github.com/googleapis/gax-go/v2 v2.1.1 // indirect
github.com/googleapis/gnostic v0.4.1 // indirect
github.com/gophercloud/gophercloud v0.24.0 // indirect
Expand Down
7 changes: 3 additions & 4 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -994,9 +994,8 @@ github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxeh
github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.2.0 h1:qJYtXnJRWmpe7m/3XlyhrsLrEURqHRM2kxzoxXqyUDs=
github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I=
github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/googleapis/gax-go v2.0.2+incompatible h1:silFMLAnr330+NRuag/VjIGF7TLp/LBrV2CJKFLWEww=
github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY=
github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
Expand Down Expand Up @@ -1033,8 +1032,8 @@ github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0U
github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/gotestyourself/gotestyourself v2.2.0+incompatible/go.mod h1:zZKM6oeNM8k+FRljX1mnzVYeS8wiGgQyvST1/GafPbY=
github.com/grafana/dskit v0.0.0-20211021180445-3bd016e9d7f1/go.mod h1:uPG2nyK4CtgNDmWv7qyzYcdI+S90kHHRWvHnBtEMBXM=
github.com/grafana/dskit v0.0.0-20220105080720-01ce9286d7d5 h1:IXo/V2+KKLYLD724qh3uRaZgAy3BV3HdtXuSs7lb3jU=
github.com/grafana/dskit v0.0.0-20220105080720-01ce9286d7d5/go.mod h1:M0/dlftwBvH7+hdNNpjMa/CUXD7gsew67mbkCuDlFXE=
github.com/grafana/dskit v0.0.0-20220209070952-ea22a8f662d0 h1:R0Pw7VjouhYSS7bsMdxEidcJbCq1KUBCzPgsh7805NM=
github.com/grafana/dskit v0.0.0-20220209070952-ea22a8f662d0/go.mod h1:Q9WmQ9cVkrHx6g4KSl6TN+N3vEOkDZd9RtyXCHd5OPQ=
github.com/grafana/go-gelf v0.0.0-20211112153804-126646b86de8 h1:aEOagXOTqtN9gd4jiDuP/5a81HdoJBqkVfn8WaxbsK4=
github.com/grafana/go-gelf v0.0.0-20211112153804-126646b86de8/go.mod h1:QAvS2C7TtQRhhv9Uf/sxD+BUhpkrPFm5jK/9MzUiDCY=
github.com/grafana/gocql v0.0.0-20200605141915-ba5dc39ece85 h1:xLuzPoOzdfNb/RF/IENCw+oLVdZB4G21VPhkHBgwSHY=
Expand Down
7 changes: 6 additions & 1 deletion pkg/distributor/distributor.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/grafana/loki/pkg/runtime"
"github.com/grafana/loki/pkg/storage/stores/shipper/compactor/retention"
"github.com/grafana/loki/pkg/tenant"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/validation"
Expand All @@ -37,7 +38,10 @@ const (
ringKey = "distributor"
)

var maxLabelCacheSize = 100000
var (
maxLabelCacheSize = 100000
rfStats = usagestats.NewInt("distributor_replication_factor")
)

// Config for a Distributor.
type Config struct {
Expand Down Expand Up @@ -168,6 +172,7 @@ func New(cfg Config, clientCfg client.Config, configs *runtime.TenantConfigs, in
}),
}
d.replicationFactor.Set(float64(ingestersRing.ReplicationFactor()))
rfStats.Set(int64(ingestersRing.ReplicationFactor()))

servs = append(servs, d.pool)
d.subservices, err = services.NewManager(servs...)
Expand Down
17 changes: 16 additions & 1 deletion pkg/ingester/flush.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/tenant"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
loki_util "github.com/grafana/loki/pkg/util"
util_log "github.com/grafana/loki/pkg/util/log"
Expand Down Expand Up @@ -90,6 +91,12 @@ var (
// 1h -> 8hr
Buckets: prometheus.LinearBuckets(1, 1, 8),
})
flushedChunksStats = usagestats.NewCounter("ingester_flushed_chunks")
flushedChunksBytesStats = usagestats.NewStatistics("ingester_flushed_chunks_bytes")
flushedChunksLinesStats = usagestats.NewStatistics("ingester_flushed_chunks_lines")
flushedChunksAgeStats = usagestats.NewStatistics("ingester_flushed_chunks_age_seconds")
flushedChunksLifespanStats = usagestats.NewStatistics("ingester_flushed_chunks_lifespan_seconds")
flushedChunksUtilizationStats = usagestats.NewStatistics("ingester_flushed_chunks_utilization")
)

const (
Expand Down Expand Up @@ -382,6 +389,7 @@ func (i *Ingester) flushChunks(ctx context.Context, fp model.Fingerprint, labelP
if err := i.store.Put(ctx, wireChunks); err != nil {
return err
}
flushedChunksStats.Inc(int64(len(wireChunks)))

// Record statistics only when actual put request did not return error.
sizePerTenant := chunkSizePerTenant.WithLabelValues(userID)
Expand All @@ -408,14 +416,21 @@ func (i *Ingester) flushChunks(ctx context.Context, fp model.Fingerprint, labelP
chunkCompressionRatio.Observe(float64(uncompressedSize) / compressedSize)
}

chunkUtilization.Observe(wc.Data.Utilization())
utilization := wc.Data.Utilization()
chunkUtilization.Observe(utilization)
chunkEntries.Observe(float64(numEntries))
chunkSize.Observe(compressedSize)
sizePerTenant.Add(compressedSize)
countPerTenant.Inc()
firstTime, lastTime := cs[i].chunk.Bounds()
chunkAge.Observe(time.Since(firstTime).Seconds())
chunkLifespan.Observe(lastTime.Sub(firstTime).Hours())

flushedChunksBytesStats.Record(compressedSize)
flushedChunksLinesStats.Record(float64(numEntries))
flushedChunksUtilizationStats.Record(utilization)
flushedChunksAgeStats.Record(time.Since(firstTime).Seconds())
flushedChunksLifespanStats.Record(lastTime.Sub(firstTime).Hours())
}

return nil
Expand Down
27 changes: 20 additions & 7 deletions pkg/ingester/ingester.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/stores/shipper"
"github.com/grafana/loki/pkg/tenant"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
errUtil "github.com/grafana/loki/pkg/util"
util_log "github.com/grafana/loki/pkg/util/log"
Expand All @@ -45,12 +46,18 @@ const (

// ErrReadOnly is returned when the ingester is shutting down and a push was
// attempted.
var ErrReadOnly = errors.New("Ingester is shutting down")

var flushQueueLength = promauto.NewGauge(prometheus.GaugeOpts{
Name: "cortex_ingester_flush_queue_length",
Help: "The total number of series pending in the flush queue.",
})
var (
ErrReadOnly = errors.New("Ingester is shutting down")

flushQueueLength = promauto.NewGauge(prometheus.GaugeOpts{
Name: "cortex_ingester_flush_queue_length",
Help: "The total number of series pending in the flush queue.",
})
compressionStats = usagestats.NewString("ingester_compression")
targetSizeStats = usagestats.NewInt("ingester_target_size_bytes")
walStats = usagestats.NewString("ingester_wal")
activeTenantsStats = usagestats.NewInt("ingester_active_tenants")
)

// Config for an ingester.
type Config struct {
Expand Down Expand Up @@ -215,7 +222,12 @@ func New(cfg Config, clientConfig client.Config, store ChunkStore, limits *valid
if cfg.ingesterClientFactory == nil {
cfg.ingesterClientFactory = client.New
}

compressionStats.Set(cfg.ChunkEncoding)
targetSizeStats.Set(int64(cfg.TargetChunkSize))
walStats.Set("disabled")
if cfg.WAL.Enabled {
walStats.Set("enabled")
}
metrics := newIngesterMetrics(registerer)

i := &Ingester{
Expand Down Expand Up @@ -549,6 +561,7 @@ func (i *Ingester) GetOrCreateInstance(instanceID string) *instance {
if !ok {
inst = newInstance(&i.cfg, instanceID, i.limiter, i.tenantConfigs, i.wal, i.metrics, i.flushOnShutdownSwitch, i.chunkFilter)
i.instances[instanceID] = inst
activeTenantsStats.Set(int64(len(i.instances)))
}
return inst
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/ingester/instance.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/grafana/loki/pkg/querier/astmapper"
"github.com/grafana/loki/pkg/runtime"
"github.com/grafana/loki/pkg/storage"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/math"
Expand Down Expand Up @@ -53,6 +54,8 @@ var (
Name: "ingester_streams_removed_total",
Help: "The total number of streams removed per tenant.",
}, []string{"tenant"})

streamsCountStats = usagestats.NewInt("ingester_streams_count")
)

type instance struct {
Expand Down Expand Up @@ -248,6 +251,7 @@ func (i *instance) createStream(pushReqStream logproto.Stream, record *WALRecord
memoryStreams.WithLabelValues(i.instanceID).Inc()
i.streamsCreatedTotal.Inc()
i.addTailersToNewStream(s)
streamsCountStats.Add(1)

if i.configs.LogStreamCreation(i.instanceID) {
level.Debug(util_log.Logger).Log(
Expand Down Expand Up @@ -288,6 +292,7 @@ func (i *instance) removeStream(s *stream) {
i.index.Delete(s.labels, s.fp)
i.streamsRemovedTotal.Inc()
memoryStreams.WithLabelValues(i.instanceID).Dec()
streamsCountStats.Add(-1)
}
}

Expand Down
5 changes: 5 additions & 0 deletions pkg/ingester/stream.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/log"
"github.com/grafana/loki/pkg/logqlmodel/stats"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util/flagext"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/validation"
Expand Down Expand Up @@ -47,6 +48,8 @@ var (

Buckets: prometheus.ExponentialBuckets(5, 2, 6),
})

chunkCreatedStats = usagestats.NewCounter("ingester_chunk_created")
)

var ErrEntriesExist = errors.New("duplicate push - entries already exist")
Expand Down Expand Up @@ -203,6 +206,7 @@ func (s *stream) Push(
chunk: s.NewChunk(),
})
chunksCreatedTotal.Inc()
chunkCreatedStats.Inc(1)
}

var storedEntries []logproto.Entry
Expand Down Expand Up @@ -379,6 +383,7 @@ func (s *stream) cutChunk(ctx context.Context) *chunkDesc {
samplesPerChunk.Observe(float64(chunk.chunk.Size()))
blocksPerChunk.Observe(float64(chunk.chunk.BlockCount()))
chunksCreatedTotal.Inc()
chunkCreatedStats.Inc(1)

s.chunks = append(s.chunks, chunkDesc{
chunk: s.NewChunk(),
Expand Down
6 changes: 6 additions & 0 deletions pkg/loghttp/push/push.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/usagestats"
"github.com/grafana/loki/pkg/util"
loki_util "github.com/grafana/loki/pkg/util"
"github.com/grafana/loki/pkg/util/unmarshal"
Expand All @@ -39,6 +40,9 @@ var (
Name: "distributor_lines_received_total",
Help: "The total number of lines received per tenant",
}, []string{"tenant"})

bytesReceivedStats = usagestats.NewCounter("distributor_bytes_received")
linesReceivedStats = usagestats.NewCounter("distributor_lines_received")
)

const applicationJSON = "application/json"
Expand Down Expand Up @@ -130,6 +134,7 @@ func ParseRequest(logger log.Logger, userID string, r *http.Request, tenantsRete
totalEntries++
entriesSize += int64(len(e.Line))
bytesIngested.WithLabelValues(userID, retentionHours).Add(float64(int64(len(e.Line))))
bytesReceivedStats.Inc(int64(len(e.Line)))
if e.Timestamp.After(mostRecentEntry) {
mostRecentEntry = e.Timestamp
}
Expand All @@ -140,6 +145,7 @@ func ParseRequest(logger log.Logger, userID string, r *http.Request, tenantsRete
if totalEntries != 0 && userID != "" {
linesIngested.WithLabelValues(userID).Add(float64(totalEntries))
}
linesReceivedStats.Inc(totalEntries)

level.Debug(logger).Log(
"msg", "push request parsed",
Expand Down
Loading

0 comments on commit bbaef79

Please sign in to comment.