mirror of https://github.com/grafana/loki
Add usage report into Loki. (#5361)
* 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>pull/5364/head
parent
02416736d0
commit
bbaef790db
@ -0,0 +1,266 @@ |
|||||||
|
package usagestats |
||||||
|
|
||||||
|
import ( |
||||||
|
"bytes" |
||||||
|
"context" |
||||||
|
"flag" |
||||||
|
"io" |
||||||
|
"math" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/go-kit/log" |
||||||
|
"github.com/go-kit/log/level" |
||||||
|
"github.com/google/uuid" |
||||||
|
"github.com/grafana/dskit/backoff" |
||||||
|
"github.com/grafana/dskit/kv" |
||||||
|
"github.com/grafana/dskit/multierror" |
||||||
|
"github.com/grafana/dskit/services" |
||||||
|
"github.com/prometheus/client_golang/prometheus" |
||||||
|
|
||||||
|
"github.com/grafana/loki/pkg/storage/chunk" |
||||||
|
"github.com/grafana/loki/pkg/util/build" |
||||||
|
) |
||||||
|
|
||||||
|
const ( |
||||||
|
// File name for the cluster seed file.
|
||||||
|
ClusterSeedFileName = "loki_cluster_seed.json" |
||||||
|
// attemptNumber how many times we will try to read a corrupted cluster seed before deleting it.
|
||||||
|
attemptNumber = 4 |
||||||
|
// seedKey is the key for the cluster seed to use with the kv store.
|
||||||
|
seedKey = "usagestats_token" |
||||||
|
) |
||||||
|
|
||||||
|
var ( |
||||||
|
reportCheckInterval = time.Minute |
||||||
|
reportInterval = 1 * time.Hour |
||||||
|
) |
||||||
|
|
||||||
|
type Config struct { |
||||||
|
Disabled bool `yaml:"disabled"` |
||||||
|
Leader bool `yaml:"-"` |
||||||
|
} |
||||||
|
|
||||||
|
// RegisterFlags adds the flags required to config this to the given FlagSet
|
||||||
|
func (cfg *Config) RegisterFlags(f *flag.FlagSet) { |
||||||
|
f.BoolVar(&cfg.Disabled, "usage-report.disabled", false, "Disable anonymous usage reporting.") |
||||||
|
} |
||||||
|
|
||||||
|
type Reporter struct { |
||||||
|
kvClient kv.Client |
||||||
|
logger log.Logger |
||||||
|
objectClient chunk.ObjectClient |
||||||
|
reg prometheus.Registerer |
||||||
|
|
||||||
|
services.Service |
||||||
|
|
||||||
|
conf Config |
||||||
|
cluster *ClusterSeed |
||||||
|
lastReport time.Time |
||||||
|
} |
||||||
|
|
||||||
|
func NewReporter(config Config, kvConfig kv.Config, objectClient chunk.ObjectClient, logger log.Logger, reg prometheus.Registerer) (*Reporter, error) { |
||||||
|
if config.Disabled { |
||||||
|
return nil, nil |
||||||
|
} |
||||||
|
kvClient, err := kv.NewClient(kvConfig, JSONCodec, kv.RegistererWithKVName(reg, "usagestats"), logger) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
r := &Reporter{ |
||||||
|
kvClient: kvClient, |
||||||
|
logger: logger, |
||||||
|
objectClient: objectClient, |
||||||
|
conf: config, |
||||||
|
reg: reg, |
||||||
|
} |
||||||
|
r.Service = services.NewBasicService(nil, r.running, nil) |
||||||
|
return r, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (rep *Reporter) initLeader(ctx context.Context) *ClusterSeed { |
||||||
|
// Try to become leader via the kv client
|
||||||
|
for backoff := backoff.New(ctx, backoff.Config{ |
||||||
|
MinBackoff: time.Second, |
||||||
|
MaxBackoff: time.Minute, |
||||||
|
MaxRetries: 0, |
||||||
|
}); ; backoff.Ongoing() { |
||||||
|
// create a new cluster seed
|
||||||
|
seed := ClusterSeed{ |
||||||
|
UID: uuid.NewString(), |
||||||
|
PrometheusVersion: build.GetVersion(), |
||||||
|
CreatedAt: time.Now(), |
||||||
|
} |
||||||
|
if err := rep.kvClient.CAS(ctx, seedKey, func(in interface{}) (out interface{}, retry bool, err error) { |
||||||
|
// The key is already set, so we don't need to do anything
|
||||||
|
if in != nil { |
||||||
|
if kvSeed, ok := in.(*ClusterSeed); ok && kvSeed.UID != seed.UID { |
||||||
|
seed = *kvSeed |
||||||
|
return nil, false, nil |
||||||
|
} |
||||||
|
} |
||||||
|
return seed, true, nil |
||||||
|
}); err != nil { |
||||||
|
level.Info(rep.logger).Log("msg", "failed to CAS cluster seed key", "err", err) |
||||||
|
continue |
||||||
|
} |
||||||
|
// Fetch the remote cluster seed.
|
||||||
|
remoteSeed, err := rep.fetchSeed(ctx, |
||||||
|
func(err error) bool { |
||||||
|
// we only want to retry if the error is not an object not found error
|
||||||
|
return !rep.objectClient.IsObjectNotFoundErr(err) |
||||||
|
}) |
||||||
|
if err != nil { |
||||||
|
if rep.objectClient.IsObjectNotFoundErr(err) { |
||||||
|
// we are the leader and we need to save the file.
|
||||||
|
if err := rep.writeSeedFile(ctx, seed); err != nil { |
||||||
|
level.Info(rep.logger).Log("msg", "failed to CAS cluster seed key", "err", err) |
||||||
|
continue |
||||||
|
} |
||||||
|
return &seed |
||||||
|
} |
||||||
|
continue |
||||||
|
} |
||||||
|
return remoteSeed |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func (rep *Reporter) init(ctx context.Context) { |
||||||
|
if rep.conf.Leader { |
||||||
|
rep.cluster = rep.initLeader(ctx) |
||||||
|
return |
||||||
|
} |
||||||
|
// follower only wait for the cluster seed to be set.
|
||||||
|
// it will try forever to fetch the cluster seed.
|
||||||
|
seed, _ := rep.fetchSeed(ctx, nil) |
||||||
|
rep.cluster = seed |
||||||
|
} |
||||||
|
|
||||||
|
// fetchSeed fetches the cluster seed from the object store and try until it succeeds.
|
||||||
|
// continueFn allow you to decide if we should continue retrying. Nil means always retry
|
||||||
|
func (rep *Reporter) fetchSeed(ctx context.Context, continueFn func(err error) bool) (*ClusterSeed, error) { |
||||||
|
var ( |
||||||
|
backoff = backoff.New(ctx, backoff.Config{ |
||||||
|
MinBackoff: time.Second, |
||||||
|
MaxBackoff: time.Minute, |
||||||
|
MaxRetries: 0, |
||||||
|
}) |
||||||
|
readingErr = 0 |
||||||
|
) |
||||||
|
for backoff.Ongoing() { |
||||||
|
seed, err := rep.readSeedFile(ctx) |
||||||
|
if err != nil { |
||||||
|
if !rep.objectClient.IsObjectNotFoundErr(err) { |
||||||
|
readingErr++ |
||||||
|
} |
||||||
|
level.Debug(rep.logger).Log("msg", "failed to read cluster seed file", "err", err) |
||||||
|
if readingErr > attemptNumber { |
||||||
|
if err := rep.objectClient.DeleteObject(ctx, ClusterSeedFileName); err != nil { |
||||||
|
level.Error(rep.logger).Log("msg", "failed to delete corrupted cluster seed file, deleting it", "err", err) |
||||||
|
} |
||||||
|
readingErr = 0 |
||||||
|
} |
||||||
|
if continueFn == nil || continueFn(err) { |
||||||
|
continue |
||||||
|
} |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
return seed, nil |
||||||
|
} |
||||||
|
return nil, backoff.Err() |
||||||
|
} |
||||||
|
|
||||||
|
// readSeedFile reads the cluster seed file from the object store.
|
||||||
|
func (rep *Reporter) readSeedFile(ctx context.Context) (*ClusterSeed, error) { |
||||||
|
reader, _, err := rep.objectClient.GetObject(ctx, ClusterSeedFileName) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
defer func() { |
||||||
|
if err := reader.Close(); err != nil { |
||||||
|
level.Error(rep.logger).Log("msg", "failed to close reader", "err", err) |
||||||
|
} |
||||||
|
}() |
||||||
|
data, err := io.ReadAll(reader) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
seed, err := JSONCodec.Decode(data) |
||||||
|
if err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
return seed.(*ClusterSeed), nil |
||||||
|
} |
||||||
|
|
||||||
|
// writeSeedFile writes the cluster seed to the object store.
|
||||||
|
func (rep *Reporter) writeSeedFile(ctx context.Context, seed ClusterSeed) error { |
||||||
|
data, err := JSONCodec.Encode(seed) |
||||||
|
if err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
return rep.objectClient.PutObject(ctx, ClusterSeedFileName, bytes.NewReader(data)) |
||||||
|
} |
||||||
|
|
||||||
|
// running inits the reporter seed and start sending report for every interval
|
||||||
|
func (rep *Reporter) running(ctx context.Context) error { |
||||||
|
rep.init(ctx) |
||||||
|
|
||||||
|
// check every minute if we should report.
|
||||||
|
ticker := time.NewTicker(reportCheckInterval) |
||||||
|
defer ticker.Stop() |
||||||
|
|
||||||
|
// find when to send the next report.
|
||||||
|
next := nextReport(reportInterval, rep.cluster.CreatedAt, time.Now()) |
||||||
|
if rep.lastReport.IsZero() { |
||||||
|
// if we never reported assumed it was the last interval.
|
||||||
|
rep.lastReport = next.Add(-reportInterval) |
||||||
|
} |
||||||
|
for { |
||||||
|
select { |
||||||
|
case <-ticker.C: |
||||||
|
now := time.Now() |
||||||
|
if !next.Equal(now) && now.Sub(rep.lastReport) < reportInterval { |
||||||
|
continue |
||||||
|
} |
||||||
|
level.Debug(rep.logger).Log("msg", "reporting cluster stats", "date", time.Now()) |
||||||
|
if err := rep.reportUsage(ctx, next); err != nil { |
||||||
|
level.Info(rep.logger).Log("msg", "failed to report usage", "err", err) |
||||||
|
continue |
||||||
|
} |
||||||
|
rep.lastReport = next |
||||||
|
next = next.Add(reportInterval) |
||||||
|
case <-ctx.Done(): |
||||||
|
return ctx.Err() |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// reportUsage reports the usage to grafana.com.
|
||||||
|
func (rep *Reporter) reportUsage(ctx context.Context, interval time.Time) error { |
||||||
|
backoff := backoff.New(ctx, backoff.Config{ |
||||||
|
MinBackoff: time.Second, |
||||||
|
MaxBackoff: 30 * time.Second, |
||||||
|
MaxRetries: 5, |
||||||
|
}) |
||||||
|
var errs multierror.MultiError |
||||||
|
for backoff.Ongoing() { |
||||||
|
if err := sendReport(ctx, rep.cluster, interval); err != nil { |
||||||
|
level.Info(rep.logger).Log("msg", "failed to send usage report", "retries", backoff.NumRetries(), "err", err) |
||||||
|
errs.Add(err) |
||||||
|
backoff.Wait() |
||||||
|
continue |
||||||
|
} |
||||||
|
level.Debug(rep.logger).Log("msg", "usage report sent with success") |
||||||
|
return nil |
||||||
|
} |
||||||
|
return errs.Err() |
||||||
|
} |
||||||
|
|
||||||
|
// nextReport compute the next report time based on the interval.
|
||||||
|
// The interval is based off the creation of the cluster seed to avoid all cluster reporting at the same time.
|
||||||
|
func nextReport(interval time.Duration, createdAt, now time.Time) time.Time { |
||||||
|
// createdAt * (x * interval ) >= now
|
||||||
|
return createdAt.Add(time.Duration(math.Ceil(float64(now.Sub(createdAt))/float64(interval))) * interval) |
||||||
|
} |
||||||
@ -0,0 +1,147 @@ |
|||||||
|
package usagestats |
||||||
|
|
||||||
|
import ( |
||||||
|
"context" |
||||||
|
"net/http" |
||||||
|
"net/http/httptest" |
||||||
|
"os" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/go-kit/log" |
||||||
|
"github.com/grafana/dskit/kv" |
||||||
|
jsoniter "github.com/json-iterator/go" |
||||||
|
"github.com/prometheus/client_golang/prometheus" |
||||||
|
"github.com/stretchr/testify/require" |
||||||
|
|
||||||
|
"github.com/grafana/loki/pkg/storage/chunk/local" |
||||||
|
"github.com/grafana/loki/pkg/storage/chunk/storage" |
||||||
|
) |
||||||
|
|
||||||
|
var metrics = storage.NewClientMetrics() |
||||||
|
|
||||||
|
func Test_LeaderElection(t *testing.T) { |
||||||
|
result := make(chan *ClusterSeed, 10) |
||||||
|
objectClient, err := storage.NewObjectClient(storage.StorageTypeFileSystem, storage.Config{ |
||||||
|
FSConfig: local.FSConfig{ |
||||||
|
Directory: t.TempDir(), |
||||||
|
}, |
||||||
|
}, metrics) |
||||||
|
require.NoError(t, err) |
||||||
|
for i := 0; i < 3; i++ { |
||||||
|
go func() { |
||||||
|
r, err := NewReporter(Config{Leader: true}, kv.Config{ |
||||||
|
Store: "inmemory", |
||||||
|
}, objectClient, log.NewLogfmtLogger(os.Stdout), prometheus.NewPedanticRegistry()) |
||||||
|
require.NoError(t, err) |
||||||
|
r.init(context.Background()) |
||||||
|
result <- r.cluster |
||||||
|
}() |
||||||
|
} |
||||||
|
for i := 0; i < 7; i++ { |
||||||
|
go func() { |
||||||
|
r, err := NewReporter(Config{Leader: false}, kv.Config{ |
||||||
|
Store: "inmemory", |
||||||
|
}, objectClient, log.NewLogfmtLogger(os.Stdout), prometheus.NewPedanticRegistry()) |
||||||
|
require.NoError(t, err) |
||||||
|
r.init(context.Background()) |
||||||
|
result <- r.cluster |
||||||
|
}() |
||||||
|
} |
||||||
|
|
||||||
|
var UID []string |
||||||
|
for i := 0; i < 10; i++ { |
||||||
|
cluster := <-result |
||||||
|
require.NotNil(t, cluster) |
||||||
|
UID = append(UID, cluster.UID) |
||||||
|
} |
||||||
|
first := UID[0] |
||||||
|
for _, uid := range UID { |
||||||
|
require.Equal(t, first, uid) |
||||||
|
} |
||||||
|
kvClient, err := kv.NewClient(kv.Config{Store: "inmemory"}, JSONCodec, prometheus.DefaultRegisterer, log.NewLogfmtLogger(os.Stdout)) |
||||||
|
require.NoError(t, err) |
||||||
|
// verify that the ID found is also correctly stored in the kv store and not overridden by another leader.
|
||||||
|
data, err := kvClient.Get(context.Background(), seedKey) |
||||||
|
require.NoError(t, err) |
||||||
|
require.Equal(t, data.(*ClusterSeed).UID, first) |
||||||
|
} |
||||||
|
|
||||||
|
func Test_ReportLoop(t *testing.T) { |
||||||
|
// stub
|
||||||
|
reportCheckInterval = 100 * time.Millisecond |
||||||
|
reportInterval = time.Second |
||||||
|
|
||||||
|
totalReport := 0 |
||||||
|
clusterIDs := []string{} |
||||||
|
server := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, r *http.Request) { |
||||||
|
var received Report |
||||||
|
totalReport++ |
||||||
|
require.NoError(t, jsoniter.NewDecoder(r.Body).Decode(&received)) |
||||||
|
clusterIDs = append(clusterIDs, received.ClusterID) |
||||||
|
rw.WriteHeader(http.StatusOK) |
||||||
|
})) |
||||||
|
usageStatsURL = server.URL |
||||||
|
|
||||||
|
objectClient, err := storage.NewObjectClient(storage.StorageTypeFileSystem, storage.Config{ |
||||||
|
FSConfig: local.FSConfig{ |
||||||
|
Directory: t.TempDir(), |
||||||
|
}, |
||||||
|
}, metrics) |
||||||
|
require.NoError(t, err) |
||||||
|
|
||||||
|
r, err := NewReporter(Config{Leader: true}, kv.Config{ |
||||||
|
Store: "inmemory", |
||||||
|
}, objectClient, log.NewLogfmtLogger(os.Stdout), prometheus.NewPedanticRegistry()) |
||||||
|
require.NoError(t, err) |
||||||
|
|
||||||
|
r.initLeader(context.Background()) |
||||||
|
ctx, cancel := context.WithCancel(context.Background()) |
||||||
|
|
||||||
|
go func() { |
||||||
|
<-time.After(6 * time.Second) |
||||||
|
cancel() |
||||||
|
}() |
||||||
|
require.Equal(t, context.Canceled, r.running(ctx)) |
||||||
|
require.GreaterOrEqual(t, totalReport, 5) |
||||||
|
first := clusterIDs[0] |
||||||
|
for _, uid := range clusterIDs { |
||||||
|
require.Equal(t, first, uid) |
||||||
|
} |
||||||
|
require.Equal(t, first, r.cluster.UID) |
||||||
|
} |
||||||
|
|
||||||
|
func Test_NextReport(t *testing.T) { |
||||||
|
fixtures := map[string]struct { |
||||||
|
interval time.Duration |
||||||
|
createdAt time.Time |
||||||
|
now time.Time |
||||||
|
|
||||||
|
next time.Time |
||||||
|
}{ |
||||||
|
"createdAt aligned with interval and now": { |
||||||
|
interval: 1 * time.Hour, |
||||||
|
createdAt: time.Unix(0, time.Hour.Nanoseconds()), |
||||||
|
now: time.Unix(0, 2*time.Hour.Nanoseconds()), |
||||||
|
next: time.Unix(0, 2*time.Hour.Nanoseconds()), |
||||||
|
}, |
||||||
|
"createdAt aligned with interval": { |
||||||
|
interval: 1 * time.Hour, |
||||||
|
createdAt: time.Unix(0, time.Hour.Nanoseconds()), |
||||||
|
now: time.Unix(0, 2*time.Hour.Nanoseconds()+1), |
||||||
|
next: time.Unix(0, 3*time.Hour.Nanoseconds()), |
||||||
|
}, |
||||||
|
"createdAt not aligned": { |
||||||
|
interval: 1 * time.Hour, |
||||||
|
createdAt: time.Unix(0, time.Hour.Nanoseconds()+18*time.Minute.Nanoseconds()+20*time.Millisecond.Nanoseconds()), |
||||||
|
now: time.Unix(0, 2*time.Hour.Nanoseconds()+1), |
||||||
|
next: time.Unix(0, 2*time.Hour.Nanoseconds()+18*time.Minute.Nanoseconds()+20*time.Millisecond.Nanoseconds()), |
||||||
|
}, |
||||||
|
} |
||||||
|
for name, f := range fixtures { |
||||||
|
t.Run(name, func(t *testing.T) { |
||||||
|
next := nextReport(f.interval, f.createdAt, f.now) |
||||||
|
require.Equal(t, f.next, next) |
||||||
|
}) |
||||||
|
} |
||||||
|
} |
||||||
@ -0,0 +1,33 @@ |
|||||||
|
package usagestats |
||||||
|
|
||||||
|
import ( |
||||||
|
"time" |
||||||
|
|
||||||
|
jsoniter "github.com/json-iterator/go" |
||||||
|
prom "github.com/prometheus/prometheus/web/api/v1" |
||||||
|
) |
||||||
|
|
||||||
|
type ClusterSeed struct { |
||||||
|
UID string `json:"UID"` |
||||||
|
CreatedAt time.Time `json:"created_at"` |
||||||
|
prom.PrometheusVersion `json:"version"` |
||||||
|
} |
||||||
|
|
||||||
|
var JSONCodec = jsonCodec{} |
||||||
|
|
||||||
|
type jsonCodec struct{} |
||||||
|
|
||||||
|
// todo we need to use the default codec for the rest of the code
|
||||||
|
// currently crashing because the in-memory kvstore use a singleton.
|
||||||
|
func (jsonCodec) Decode(data []byte) (interface{}, error) { |
||||||
|
var seed ClusterSeed |
||||||
|
if err := jsoniter.ConfigFastest.Unmarshal(data, &seed); err != nil { |
||||||
|
return nil, err |
||||||
|
} |
||||||
|
return &seed, nil |
||||||
|
} |
||||||
|
|
||||||
|
func (jsonCodec) Encode(obj interface{}) ([]byte, error) { |
||||||
|
return jsoniter.ConfigFastest.Marshal(obj) |
||||||
|
} |
||||||
|
func (jsonCodec) CodecID() string { return "usagestats.jsonCodec" } |
||||||
@ -0,0 +1,352 @@ |
|||||||
|
package usagestats |
||||||
|
|
||||||
|
import ( |
||||||
|
"bytes" |
||||||
|
"context" |
||||||
|
"encoding/json" |
||||||
|
"expvar" |
||||||
|
"fmt" |
||||||
|
"io" |
||||||
|
"math" |
||||||
|
"net/http" |
||||||
|
"runtime" |
||||||
|
"strings" |
||||||
|
"sync" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/grafana/loki/pkg/util/build" |
||||||
|
|
||||||
|
"github.com/cespare/xxhash/v2" |
||||||
|
jsoniter "github.com/json-iterator/go" |
||||||
|
prom "github.com/prometheus/prometheus/web/api/v1" |
||||||
|
"go.uber.org/atomic" |
||||||
|
) |
||||||
|
|
||||||
|
var ( |
||||||
|
httpClient = http.Client{Timeout: 5 * time.Second} |
||||||
|
usageStatsURL = "https://stats.grafana.org/loki-usage-report" |
||||||
|
statsPrefix = "github.com/grafana/loki/" |
||||||
|
targetKey = "target" |
||||||
|
editionKey = "edition" |
||||||
|
) |
||||||
|
|
||||||
|
// Report is the JSON object sent to the stats server
|
||||||
|
type Report struct { |
||||||
|
ClusterID string `json:"clusterID"` |
||||||
|
CreatedAt time.Time `json:"createdAt"` |
||||||
|
Interval time.Time `json:"interval"` |
||||||
|
Target string `json:"target"` |
||||||
|
prom.PrometheusVersion `json:"version"` |
||||||
|
Os string `json:"os"` |
||||||
|
Arch string `json:"arch"` |
||||||
|
Edition string `json:"edition"` |
||||||
|
Metrics map[string]interface{} `json:"metrics"` |
||||||
|
} |
||||||
|
|
||||||
|
// sendReport sends the report to the stats server
|
||||||
|
func sendReport(ctx context.Context, seed *ClusterSeed, interval time.Time) error { |
||||||
|
report := buildReport(seed, interval) |
||||||
|
out, err := jsoniter.MarshalIndent(report, "", " ") |
||||||
|
if err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
req, err := http.NewRequest(http.MethodPost, usageStatsURL, bytes.NewBuffer(out)) |
||||||
|
if err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
req.Header.Set("Content-Type", "application/json") |
||||||
|
resp, err := httpClient.Do(req.WithContext(ctx)) |
||||||
|
if err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
defer resp.Body.Close() |
||||||
|
if resp.StatusCode/100 != 2 { |
||||||
|
data, err := io.ReadAll(resp.Body) |
||||||
|
if err != nil { |
||||||
|
return err |
||||||
|
} |
||||||
|
return fmt.Errorf("failed to send usage stats: %s body: %s", resp.Status, string(data)) |
||||||
|
} |
||||||
|
return nil |
||||||
|
} |
||||||
|
|
||||||
|
// buildReport builds the report to be sent to the stats server
|
||||||
|
func buildReport(seed *ClusterSeed, interval time.Time) Report { |
||||||
|
var ( |
||||||
|
targetName string |
||||||
|
editionName string |
||||||
|
) |
||||||
|
if target := expvar.Get(statsPrefix + targetKey); target != nil { |
||||||
|
if target, ok := target.(*expvar.String); ok { |
||||||
|
targetName = target.Value() |
||||||
|
} |
||||||
|
} |
||||||
|
if edition := expvar.Get(statsPrefix + editionKey); edition != nil { |
||||||
|
if edition, ok := edition.(*expvar.String); ok { |
||||||
|
editionName = edition.Value() |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
return Report{ |
||||||
|
ClusterID: seed.UID, |
||||||
|
PrometheusVersion: build.GetVersion(), |
||||||
|
CreatedAt: seed.CreatedAt, |
||||||
|
Interval: interval, |
||||||
|
Os: runtime.GOOS, |
||||||
|
Arch: runtime.GOARCH, |
||||||
|
Target: targetName, |
||||||
|
Edition: editionName, |
||||||
|
Metrics: buildMetrics(), |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// buildMetrics builds the metrics part of the report to be sent to the stats server
|
||||||
|
func buildMetrics() map[string]interface{} { |
||||||
|
result := map[string]interface{}{ |
||||||
|
"memstats": memstats(), |
||||||
|
"num_cpu": runtime.NumCPU(), |
||||||
|
"num_goroutine": runtime.NumGoroutine(), |
||||||
|
} |
||||||
|
expvar.Do(func(kv expvar.KeyValue) { |
||||||
|
if !strings.HasPrefix(kv.Key, statsPrefix) || kv.Key == statsPrefix+targetKey || kv.Key == statsPrefix+editionKey { |
||||||
|
return |
||||||
|
} |
||||||
|
var value interface{} |
||||||
|
switch v := kv.Value.(type) { |
||||||
|
case *expvar.Int: |
||||||
|
value = v.Value() |
||||||
|
case *expvar.Float: |
||||||
|
value = v.Value() |
||||||
|
case *expvar.String: |
||||||
|
value = v.Value() |
||||||
|
case *Statistics: |
||||||
|
value = v.Value() |
||||||
|
case *Counter: |
||||||
|
v.updateRate() |
||||||
|
value = v.Value() |
||||||
|
v.reset() |
||||||
|
case *WordCounter: |
||||||
|
value = v.Value() |
||||||
|
default: |
||||||
|
value = v.String() |
||||||
|
} |
||||||
|
result[strings.TrimPrefix(kv.Key, statsPrefix)] = value |
||||||
|
}) |
||||||
|
return result |
||||||
|
} |
||||||
|
|
||||||
|
func memstats() interface{} { |
||||||
|
stats := new(runtime.MemStats) |
||||||
|
runtime.ReadMemStats(stats) |
||||||
|
return map[string]interface{}{ |
||||||
|
"alloc": stats.Alloc, |
||||||
|
"total_alloc": stats.TotalAlloc, |
||||||
|
"sys": stats.Sys, |
||||||
|
"heap_alloc": stats.HeapAlloc, |
||||||
|
"heap_inuse": stats.HeapInuse, |
||||||
|
"stack_inuse": stats.StackInuse, |
||||||
|
"pause_total_ns": stats.PauseTotalNs, |
||||||
|
"num_gc": stats.NumGC, |
||||||
|
"gc_cpu_fraction": stats.GCCPUFraction, |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// NewFloat returns a new Float stats object.
|
||||||
|
func NewFloat(name string) *expvar.Float { |
||||||
|
return expvar.NewFloat(statsPrefix + name) |
||||||
|
} |
||||||
|
|
||||||
|
// NewInt returns a new Int stats object.
|
||||||
|
func NewInt(name string) *expvar.Int { |
||||||
|
return expvar.NewInt(statsPrefix + name) |
||||||
|
} |
||||||
|
|
||||||
|
// NewString returns a new String stats object.
|
||||||
|
func NewString(name string) *expvar.String { |
||||||
|
return expvar.NewString(statsPrefix + name) |
||||||
|
} |
||||||
|
|
||||||
|
// Target sets the target name.
|
||||||
|
func Target(target string) { |
||||||
|
NewString(targetKey).Set(target) |
||||||
|
} |
||||||
|
|
||||||
|
// Edition sets the edition name.
|
||||||
|
func Edition(edition string) { |
||||||
|
NewString(editionKey).Set(edition) |
||||||
|
} |
||||||
|
|
||||||
|
type Statistics struct { |
||||||
|
min *atomic.Float64 |
||||||
|
max *atomic.Float64 |
||||||
|
count *atomic.Int64 |
||||||
|
|
||||||
|
avg *atomic.Float64 |
||||||
|
|
||||||
|
// require for stddev and stdvar
|
||||||
|
mean *atomic.Float64 |
||||||
|
value *atomic.Float64 |
||||||
|
} |
||||||
|
|
||||||
|
// NewStatistics returns a new Statistics object.
|
||||||
|
// Statistics object is thread-safe and compute statistics on the fly based on sample recorded.
|
||||||
|
// Available statistics are:
|
||||||
|
// - min
|
||||||
|
// - max
|
||||||
|
// - avg
|
||||||
|
// - count
|
||||||
|
// - stddev
|
||||||
|
// - stdvar
|
||||||
|
func NewStatistics(name string) *Statistics { |
||||||
|
s := &Statistics{ |
||||||
|
min: atomic.NewFloat64(math.Inf(0)), |
||||||
|
max: atomic.NewFloat64(math.Inf(-1)), |
||||||
|
count: atomic.NewInt64(0), |
||||||
|
avg: atomic.NewFloat64(0), |
||||||
|
mean: atomic.NewFloat64(0), |
||||||
|
value: atomic.NewFloat64(0), |
||||||
|
} |
||||||
|
expvar.Publish(statsPrefix+name, s) |
||||||
|
return s |
||||||
|
} |
||||||
|
|
||||||
|
func (s *Statistics) String() string { |
||||||
|
b, _ := json.Marshal(s.Value()) |
||||||
|
return string(b) |
||||||
|
} |
||||||
|
|
||||||
|
func (s *Statistics) Value() map[string]interface{} { |
||||||
|
stdvar := s.value.Load() / float64(s.count.Load()) |
||||||
|
stddev := math.Sqrt(stdvar) |
||||||
|
min := s.min.Load() |
||||||
|
max := s.max.Load() |
||||||
|
result := map[string]interface{}{ |
||||||
|
"avg": s.avg.Load(), |
||||||
|
"count": s.count.Load(), |
||||||
|
} |
||||||
|
if !math.IsInf(min, 0) { |
||||||
|
result["min"] = min |
||||||
|
} |
||||||
|
if !math.IsInf(max, 0) { |
||||||
|
result["max"] = s.max.Load() |
||||||
|
} |
||||||
|
if !math.IsNaN(stddev) { |
||||||
|
result["stddev"] = stddev |
||||||
|
} |
||||||
|
if !math.IsNaN(stdvar) { |
||||||
|
result["stdvar"] = stdvar |
||||||
|
} |
||||||
|
return result |
||||||
|
} |
||||||
|
|
||||||
|
func (s *Statistics) Record(v float64) { |
||||||
|
for { |
||||||
|
min := s.min.Load() |
||||||
|
if min <= v { |
||||||
|
break |
||||||
|
} |
||||||
|
if s.min.CAS(min, v) { |
||||||
|
break |
||||||
|
} |
||||||
|
} |
||||||
|
for { |
||||||
|
max := s.max.Load() |
||||||
|
if max >= v { |
||||||
|
break |
||||||
|
} |
||||||
|
if s.max.CAS(max, v) { |
||||||
|
break |
||||||
|
} |
||||||
|
} |
||||||
|
for { |
||||||
|
avg := s.avg.Load() |
||||||
|
count := s.count.Load() |
||||||
|
mean := s.mean.Load() |
||||||
|
value := s.value.Load() |
||||||
|
|
||||||
|
delta := v - mean |
||||||
|
newCount := count + 1 |
||||||
|
newMean := mean + (delta / float64(newCount)) |
||||||
|
newValue := value + (delta * (v - newMean)) |
||||||
|
newAvg := avg + ((v - avg) / float64(newCount)) |
||||||
|
if s.avg.CAS(avg, newAvg) && s.count.CAS(count, newCount) && s.mean.CAS(mean, newMean) && s.value.CAS(value, newValue) { |
||||||
|
break |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
type Counter struct { |
||||||
|
total *atomic.Int64 |
||||||
|
rate *atomic.Float64 |
||||||
|
|
||||||
|
resetTime time.Time |
||||||
|
} |
||||||
|
|
||||||
|
// NewCounter returns a new Counter stats object.
|
||||||
|
func NewCounter(name string) *Counter { |
||||||
|
c := &Counter{ |
||||||
|
total: atomic.NewInt64(0), |
||||||
|
rate: atomic.NewFloat64(0), |
||||||
|
resetTime: time.Now(), |
||||||
|
} |
||||||
|
expvar.Publish(statsPrefix+name, c) |
||||||
|
return c |
||||||
|
} |
||||||
|
|
||||||
|
func (c *Counter) updateRate() { |
||||||
|
total := c.total.Load() |
||||||
|
c.rate.Store(float64(total) / time.Since(c.resetTime).Seconds()) |
||||||
|
} |
||||||
|
|
||||||
|
func (c *Counter) reset() { |
||||||
|
c.total.Store(0) |
||||||
|
c.rate.Store(0) |
||||||
|
c.resetTime = time.Now() |
||||||
|
} |
||||||
|
|
||||||
|
func (c *Counter) Inc(i int64) { |
||||||
|
c.total.Add(i) |
||||||
|
} |
||||||
|
|
||||||
|
func (c *Counter) String() string { |
||||||
|
b, _ := json.Marshal(c.Value()) |
||||||
|
return string(b) |
||||||
|
} |
||||||
|
|
||||||
|
func (c *Counter) Value() map[string]interface{} { |
||||||
|
return map[string]interface{}{ |
||||||
|
"total": c.total.Load(), |
||||||
|
"rate": c.rate.Load(), |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
type WordCounter struct { |
||||||
|
words sync.Map |
||||||
|
count *atomic.Int64 |
||||||
|
} |
||||||
|
|
||||||
|
// NewWordCounter returns a new WordCounter stats object.
|
||||||
|
// WordCounter object is thread-safe and count the amount of word recorded.
|
||||||
|
func NewWordCounter(name string) *WordCounter { |
||||||
|
c := &WordCounter{ |
||||||
|
count: atomic.NewInt64(0), |
||||||
|
words: sync.Map{}, |
||||||
|
} |
||||||
|
expvar.Publish(statsPrefix+name, c) |
||||||
|
return c |
||||||
|
} |
||||||
|
|
||||||
|
func (w *WordCounter) Add(word string) { |
||||||
|
if _, loaded := w.words.LoadOrStore(xxhash.Sum64String(word), struct{}{}); !loaded { |
||||||
|
w.count.Add(1) |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
func (w *WordCounter) String() string { |
||||||
|
b, _ := json.Marshal(w.Value()) |
||||||
|
return string(b) |
||||||
|
} |
||||||
|
|
||||||
|
func (w *WordCounter) Value() int64 { |
||||||
|
return w.count.Load() |
||||||
|
} |
||||||
@ -0,0 +1,97 @@ |
|||||||
|
package usagestats |
||||||
|
|
||||||
|
import ( |
||||||
|
"runtime" |
||||||
|
"testing" |
||||||
|
"time" |
||||||
|
|
||||||
|
"github.com/grafana/loki/pkg/util/build" |
||||||
|
|
||||||
|
"github.com/google/uuid" |
||||||
|
jsoniter "github.com/json-iterator/go" |
||||||
|
"github.com/stretchr/testify/require" |
||||||
|
) |
||||||
|
|
||||||
|
func Test_BuildReport(t *testing.T) { |
||||||
|
now := time.Now() |
||||||
|
seed := &ClusterSeed{ |
||||||
|
UID: uuid.New().String(), |
||||||
|
CreatedAt: now, |
||||||
|
} |
||||||
|
|
||||||
|
Edition("OSS") |
||||||
|
Target("compactor") |
||||||
|
NewString("compression").Set("lz4") |
||||||
|
NewInt("compression_ratio").Set(100) |
||||||
|
NewFloat("size_mb").Set(100.1) |
||||||
|
NewCounter("lines_written").Inc(200) |
||||||
|
s := NewStatistics("query_throughput") |
||||||
|
s.Record(300) |
||||||
|
s.Record(5) |
||||||
|
w := NewWordCounter("active_tenants") |
||||||
|
w.Add("foo") |
||||||
|
w.Add("bar") |
||||||
|
w.Add("foo") |
||||||
|
|
||||||
|
r := buildReport(seed, now.Add(time.Hour)) |
||||||
|
require.Equal(t, r.Arch, runtime.GOARCH) |
||||||
|
require.Equal(t, r.Os, runtime.GOOS) |
||||||
|
require.Equal(t, r.PrometheusVersion, build.GetVersion()) |
||||||
|
require.Equal(t, r.Edition, "OSS") |
||||||
|
require.Equal(t, r.Target, "compactor") |
||||||
|
require.Equal(t, r.Metrics["num_cpu"], runtime.NumCPU()) |
||||||
|
require.Equal(t, r.Metrics["num_goroutine"], runtime.NumGoroutine()) |
||||||
|
require.Equal(t, r.Metrics["compression"], "lz4") |
||||||
|
require.Equal(t, r.Metrics["compression_ratio"], int64(100)) |
||||||
|
require.Equal(t, r.Metrics["size_mb"], 100.1) |
||||||
|
require.Equal(t, r.Metrics["lines_written"].(map[string]interface{})["total"], int64(200)) |
||||||
|
require.Equal(t, r.Metrics["query_throughput"].(map[string]interface{})["min"], float64(5)) |
||||||
|
require.Equal(t, r.Metrics["query_throughput"].(map[string]interface{})["max"], float64(300)) |
||||||
|
require.Equal(t, r.Metrics["query_throughput"].(map[string]interface{})["count"], int64(2)) |
||||||
|
require.Equal(t, r.Metrics["query_throughput"].(map[string]interface{})["avg"], float64(300+5)/2) |
||||||
|
require.Equal(t, r.Metrics["active_tenants"], int64(2)) |
||||||
|
|
||||||
|
out, _ := jsoniter.MarshalIndent(r, "", " ") |
||||||
|
t.Log(string(out)) |
||||||
|
} |
||||||
|
|
||||||
|
func TestCounter(t *testing.T) { |
||||||
|
c := NewCounter("test_counter") |
||||||
|
c.Inc(100) |
||||||
|
c.Inc(200) |
||||||
|
c.Inc(300) |
||||||
|
time.Sleep(1 * time.Second) |
||||||
|
c.updateRate() |
||||||
|
v := c.Value() |
||||||
|
require.Equal(t, int64(600), v["total"]) |
||||||
|
require.GreaterOrEqual(t, v["rate"], float64(590)) |
||||||
|
c.reset() |
||||||
|
require.Equal(t, int64(0), c.Value()["total"]) |
||||||
|
require.Equal(t, float64(0), c.Value()["rate"]) |
||||||
|
} |
||||||
|
|
||||||
|
func TestStatistic(t *testing.T) { |
||||||
|
s := NewStatistics("test_stats") |
||||||
|
s.Record(100) |
||||||
|
s.Record(200) |
||||||
|
s.Record(300) |
||||||
|
v := s.Value() |
||||||
|
require.Equal(t, float64(100), v["min"]) |
||||||
|
require.Equal(t, float64(300), v["max"]) |
||||||
|
require.Equal(t, int64(3), v["count"]) |
||||||
|
require.Equal(t, float64(100+200+300)/3, v["avg"]) |
||||||
|
require.Equal(t, float64(81.64965809277261), v["stddev"]) |
||||||
|
require.Equal(t, float64(6666.666666666667), v["stdvar"]) |
||||||
|
} |
||||||
|
|
||||||
|
func TestWordCounter(t *testing.T) { |
||||||
|
w := NewWordCounter("test_words_count") |
||||||
|
for i := 0; i < 100; i++ { |
||||||
|
go func() { |
||||||
|
w.Add("foo") |
||||||
|
w.Add("bar") |
||||||
|
w.Add("foo") |
||||||
|
}() |
||||||
|
} |
||||||
|
require.Equal(t, int64(2), w.Value()) |
||||||
|
} |
||||||
@ -1,118 +0,0 @@ |
|||||||
// Copyright 2021 Google Inc. All rights reserved.
|
|
||||||
// Use of this source code is governed by a BSD-style
|
|
||||||
// license that can be found in the LICENSE file.
|
|
||||||
|
|
||||||
package uuid |
|
||||||
|
|
||||||
import ( |
|
||||||
"bytes" |
|
||||||
"database/sql/driver" |
|
||||||
"encoding/json" |
|
||||||
"fmt" |
|
||||||
) |
|
||||||
|
|
||||||
var jsonNull = []byte("null") |
|
||||||
|
|
||||||
// NullUUID represents a UUID that may be null.
|
|
||||||
// NullUUID implements the SQL driver.Scanner interface so
|
|
||||||
// it can be used as a scan destination:
|
|
||||||
//
|
|
||||||
// var u uuid.NullUUID
|
|
||||||
// err := db.QueryRow("SELECT name FROM foo WHERE id=?", id).Scan(&u)
|
|
||||||
// ...
|
|
||||||
// if u.Valid {
|
|
||||||
// // use u.UUID
|
|
||||||
// } else {
|
|
||||||
// // NULL value
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
type NullUUID struct { |
|
||||||
UUID UUID |
|
||||||
Valid bool // Valid is true if UUID is not NULL
|
|
||||||
} |
|
||||||
|
|
||||||
// Scan implements the SQL driver.Scanner interface.
|
|
||||||
func (nu *NullUUID) Scan(value interface{}) error { |
|
||||||
if value == nil { |
|
||||||
nu.UUID, nu.Valid = Nil, false |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
err := nu.UUID.Scan(value) |
|
||||||
if err != nil { |
|
||||||
nu.Valid = false |
|
||||||
return err |
|
||||||
} |
|
||||||
|
|
||||||
nu.Valid = true |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// Value implements the driver Valuer interface.
|
|
||||||
func (nu NullUUID) Value() (driver.Value, error) { |
|
||||||
if !nu.Valid { |
|
||||||
return nil, nil |
|
||||||
} |
|
||||||
// Delegate to UUID Value function
|
|
||||||
return nu.UUID.Value() |
|
||||||
} |
|
||||||
|
|
||||||
// MarshalBinary implements encoding.BinaryMarshaler.
|
|
||||||
func (nu NullUUID) MarshalBinary() ([]byte, error) { |
|
||||||
if nu.Valid { |
|
||||||
return nu.UUID[:], nil |
|
||||||
} |
|
||||||
|
|
||||||
return []byte(nil), nil |
|
||||||
} |
|
||||||
|
|
||||||
// UnmarshalBinary implements encoding.BinaryUnmarshaler.
|
|
||||||
func (nu *NullUUID) UnmarshalBinary(data []byte) error { |
|
||||||
if len(data) != 16 { |
|
||||||
return fmt.Errorf("invalid UUID (got %d bytes)", len(data)) |
|
||||||
} |
|
||||||
copy(nu.UUID[:], data) |
|
||||||
nu.Valid = true |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// MarshalText implements encoding.TextMarshaler.
|
|
||||||
func (nu NullUUID) MarshalText() ([]byte, error) { |
|
||||||
if nu.Valid { |
|
||||||
return nu.UUID.MarshalText() |
|
||||||
} |
|
||||||
|
|
||||||
return jsonNull, nil |
|
||||||
} |
|
||||||
|
|
||||||
// UnmarshalText implements encoding.TextUnmarshaler.
|
|
||||||
func (nu *NullUUID) UnmarshalText(data []byte) error { |
|
||||||
id, err := ParseBytes(data) |
|
||||||
if err != nil { |
|
||||||
nu.Valid = false |
|
||||||
return err |
|
||||||
} |
|
||||||
nu.UUID = id |
|
||||||
nu.Valid = true |
|
||||||
return nil |
|
||||||
} |
|
||||||
|
|
||||||
// MarshalJSON implements json.Marshaler.
|
|
||||||
func (nu NullUUID) MarshalJSON() ([]byte, error) { |
|
||||||
if nu.Valid { |
|
||||||
return json.Marshal(nu.UUID) |
|
||||||
} |
|
||||||
|
|
||||||
return jsonNull, nil |
|
||||||
} |
|
||||||
|
|
||||||
// UnmarshalJSON implements json.Unmarshaler.
|
|
||||||
func (nu *NullUUID) UnmarshalJSON(data []byte) error { |
|
||||||
if bytes.Equal(data, jsonNull) { |
|
||||||
*nu = NullUUID{} |
|
||||||
return nil // valid null UUID
|
|
||||||
} |
|
||||||
err := json.Unmarshal(data, &nu.UUID) |
|
||||||
nu.Valid = err == nil |
|
||||||
return err |
|
||||||
} |
|
||||||
Loading…
Reference in new issue