Loki: Improve spans usage (#8927)

**What this PR does / why we need it**:
- At different places, inherit the span/spanlogger from the given
context instead of instantiating a new one from scratch, which fix spans
being orphaned on a read/write operation.
- At different places, turn spans into events. Events are lighter than
spans and by having fewer spans in the trace, trace visualization will
be cleaner without losing any details.
- Adds new spans/events to places that might be a bottleneck for our
writes/reads.
integrate-laser
Dylan Guedes 3 years ago committed by GitHub
parent de84737368
commit 9159c1dac3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      docs/sources/configuration/_index.md
  2. 19
      pkg/distributor/distributor.go
  3. 68
      pkg/distributor/ratestore.go
  4. 12
      pkg/ingester/stream.go
  5. 5
      pkg/logql/engine.go
  6. 7
      pkg/querier/http.go
  7. 8
      pkg/querier/queryrange/downstreamer.go
  8. 17
      pkg/querier/queryrange/limits.go
  9. 3
      pkg/querier/queryrange/log_result_cache.go
  10. 4
      pkg/querier/queryrange/queryrangebase/query_range.go
  11. 10
      pkg/querier/queryrange/queryrangebase/results_cache.go
  12. 15
      pkg/querier/queryrange/shard_resolver.go
  13. 5
      pkg/storage/chunk/client/util/parallel_chunk_fetch.go
  14. 5
      pkg/storage/chunk/fetcher/fetcher.go
  15. 17
      pkg/storage/stores/composite_store_entry.go
  16. 17
      pkg/storage/stores/series/series_index_store.go
  17. 5
      pkg/storage/stores/series_store_write.go
  18. 7
      pkg/storage/stores/tsdb/index_client.go

@ -458,6 +458,10 @@ rate_store:
# updating rates
# CLI flag: -distributor.rate-store.ingester-request-timeout
[ingester_request_timeout: <duration> | default = 500ms]
# If enabled, detailed logs and spans will be emitted.
# CLI flag: -distributor.rate-store.debug
[debug: <boolean> | default = false]
```
### querier

@ -284,7 +284,13 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
validationContext := d.validator.getValidationContextForTime(time.Now(), tenantID)
func() {
sp, _ := opentracing.StartSpanFromContext(ctx, "distributor.ValidatePushRequest")
sp := opentracing.SpanFromContext(ctx)
if sp != nil {
sp.LogKV("event", "start to validate request")
defer func() {
sp.LogKV("event", "finished to validate request")
}()
}
for _, stream := range req.Streams {
// Return early if stream does not contain any entries
if len(stream.Entries) == 0 {
@ -345,7 +351,6 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
streams = append(streams, streamTracker{stream: stream})
}
}
sp.Finish()
}()
// Return early if none of the streams contained entries
@ -368,8 +373,14 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
ingesterDescs := map[string]ring.InstanceDesc{}
if err := func() error {
sp, _ := opentracing.StartSpanFromContext(ctx, "distributor.QueryIngestersRing")
defer sp.Finish()
sp := opentracing.SpanFromContext(ctx)
if sp != nil {
sp.LogKV("event", "started to query ingesters ring")
defer func() {
sp.LogKV("event", "finished to query ingesters ring")
}()
}
for i, key := range keys {
replicationSet, err := d.ingestersRing.Get(key, ring.WriteNoExtend, descs[:0], nil, nil)
if err != nil {

@ -7,6 +7,7 @@ import (
"time"
"github.com/grafana/loki/pkg/util"
"github.com/opentracing/opentracing-go"
"github.com/weaveworks/common/instrument"
@ -31,12 +32,14 @@ type RateStoreConfig struct {
MaxParallelism int `yaml:"max_request_parallelism"`
StreamRateUpdateInterval time.Duration `yaml:"stream_rate_update_interval"`
IngesterReqTimeout time.Duration `yaml:"ingester_request_timeout"`
Debug bool `yaml:"debug"`
}
func (cfg *RateStoreConfig) RegisterFlagsWithPrefix(prefix string, fs *flag.FlagSet) {
fs.IntVar(&cfg.MaxParallelism, prefix+".max-request-parallelism", 200, "The max number of concurrent requests to make to ingester stream apis")
fs.DurationVar(&cfg.StreamRateUpdateInterval, prefix+".stream-rate-update-interval", time.Second, "The interval on which distributors will update current stream rates from ingesters")
fs.DurationVar(&cfg.IngesterReqTimeout, prefix+".ingester-request-timeout", 500*time.Millisecond, "Timeout for communication between distributors and any given ingester when updating rates")
fs.BoolVar(&cfg.Debug, prefix+".debug", false, "If enabled, detailed logs and spans will be emitted.")
}
type ingesterClient struct {
@ -63,6 +66,8 @@ type rateStore struct {
limits Limits
metrics *ratestoreMetrics
debug bool
}
func NewRateStore(cfg RateStoreConfig, r ring.ReadRing, cf poolClientFactory, l Limits, registerer prometheus.Registerer) *rateStore { //nolint
@ -75,6 +80,7 @@ func NewRateStore(cfg RateStoreConfig, r ring.ReadRing, cf poolClientFactory, l
limits: l,
metrics: newRateStoreMetrics(registerer),
rates: make(map[string]map[uint64]expiringRate),
debug: cfg.Debug,
}
rateCollectionInterval := util.DurationWithJitter(cfg.StreamRateUpdateInterval, 0.2)
@ -94,7 +100,7 @@ func (s *rateStore) instrumentedUpdateAllRates(ctx context.Context) error {
}
func (s *rateStore) updateAllRates(ctx context.Context) error {
clients, err := s.getClients()
clients, err := s.getClients(ctx)
if err != nil {
level.Error(util_log.Logger).Log("msg", "error getting ingester clients", "err", err)
s.metrics.rateRefreshFailures.WithLabelValues("ring").Inc()
@ -102,8 +108,8 @@ func (s *rateStore) updateAllRates(ctx context.Context) error {
}
streamRates := s.getRates(ctx, clients)
updated := s.aggregateByShard(streamRates)
updateStats := s.updateRates(updated)
updated := s.aggregateByShard(ctx, streamRates)
updateStats := s.updateRates(ctx, updated)
s.metrics.maxStreamRate.Set(float64(updateStats.maxRate))
s.metrics.maxStreamShardCount.Set(float64(updateStats.maxShards))
@ -120,7 +126,13 @@ type rateStats struct {
expiredCount int64
}
func (s *rateStore) updateRates(updated map[string]map[uint64]expiringRate) rateStats {
func (s *rateStore) updateRates(ctx context.Context, updated map[string]map[uint64]expiringRate) rateStats {
if s.debug {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("event", "started to update rates")
defer sp.LogKV("event", "finished to update rates")
}
}
s.rateLock.Lock()
defer s.rateLock.Unlock()
@ -179,7 +191,13 @@ func (s *rateStore) anyShardingEnabled() bool {
return false
}
func (s *rateStore) aggregateByShard(streamRates map[string]map[uint64]*logproto.StreamRate) map[string]map[uint64]expiringRate {
func (s *rateStore) aggregateByShard(ctx context.Context, streamRates map[string]map[uint64]*logproto.StreamRate) map[string]map[uint64]expiringRate {
if s.debug {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("started to aggregate by shard")
defer sp.LogKV("finished to aggregate by shard")
}
}
rates := map[string]map[uint64]expiringRate{}
for tID, tenant := range streamRates {
@ -208,6 +226,13 @@ func max(a, b int64) int64 {
}
func (s *rateStore) getRates(ctx context.Context, clients []ingesterClient) map[string]map[uint64]*logproto.StreamRate {
if s.debug {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("event", "started to get rates from ingesters")
defer sp.LogKV("event", "finished to get rates from ingesters")
}
}
parallelClients := make(chan ingesterClient, len(clients))
responses := make(chan *logproto.StreamRatesResponse, len(clients))
@ -225,16 +250,24 @@ func (s *rateStore) getRates(ctx context.Context, clients []ingesterClient) map[
func (s *rateStore) getRatesFromIngesters(ctx context.Context, clients chan ingesterClient, responses chan *logproto.StreamRatesResponse) {
for c := range clients {
ctx, cancel := context.WithTimeout(ctx, s.ingesterTimeout)
func() {
if s.debug {
startTime := time.Now()
defer func() {
level.Debug(util_log.Logger).Log("msg", "get rates from ingester", "duration", time.Since(startTime), "ingester", c.addr)
}()
}
ctx, cancel := context.WithTimeout(ctx, s.ingesterTimeout)
resp, err := c.client.GetStreamRates(ctx, &logproto.StreamRatesRequest{})
if err != nil {
level.Error(util_log.Logger).Log("msg", "unable to get stream rates from ingester", "ingester", c.addr, "err", err)
s.metrics.rateRefreshFailures.WithLabelValues(c.addr).Inc()
}
resp, err := c.client.GetStreamRates(ctx, &logproto.StreamRatesRequest{})
if err != nil {
level.Error(util_log.Logger).Log("msg", "unable to get stream rates from ingester", "ingester", c.addr, "err", err)
s.metrics.rateRefreshFailures.WithLabelValues(c.addr).Inc()
}
responses <- resp
cancel()
responses <- resp
cancel()
}()
}
}
@ -269,7 +302,14 @@ func (s *rateStore) ratesPerStream(responses chan *logproto.StreamRatesResponse,
return streamRates
}
func (s *rateStore) getClients() ([]ingesterClient, error) {
func (s *rateStore) getClients(ctx context.Context) ([]ingesterClient, error) {
if s.debug {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("event", "ratestore started getting clients")
defer sp.LogKV("event", "ratestore finished getting clients")
}
}
ingesters, err := s.ring.GetAllHealthy(ring.Read)
if err != nil {
return nil, err

@ -295,8 +295,11 @@ func (s *stream) recordAndSendToTailers(record *wal.Record, entries []logproto.E
}
func (s *stream) storeEntries(ctx context.Context, entries []logproto.Entry) (int, []logproto.Entry, []entryWithError) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "stream.storeEntries")
defer sp.Finish()
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("event", "stream started to store entries", "labels", s.labelsString)
defer sp.LogKV("event", "stream finished to store entries")
}
var bytesAdded, outOfOrderSamples, outOfOrderBytes int
var invalid []entryWithError
@ -327,7 +330,6 @@ func (s *stream) storeEntries(ctx context.Context, entries []logproto.Entry) (in
bytesAdded += len(entries[i].Line)
storedEntries = append(storedEntries, entries[i])
}
s.reportMetrics(outOfOrderSamples, outOfOrderBytes, 0, 0)
return bytesAdded, storedEntries, invalid
}
@ -423,6 +425,10 @@ func (s *stream) reportMetrics(outOfOrderSamples, outOfOrderBytes, rateLimitedSa
}
func (s *stream) cutChunk(ctx context.Context) *chunkDesc {
if sp := opentracing.SpanFromContext(ctx); sp != nil {
sp.LogKV("event", "stream started to cut chunk")
defer sp.LogKV("event", "stream finished to cut chunk")
}
// If the chunk has no more space call Close to make sure anything in the head block is cut and compressed
chunk := &s.chunks[len(s.chunks)-1]
err := chunk.chunk.Close()

@ -11,6 +11,7 @@ import (
"time"
"github.com/grafana/loki/pkg/logqlmodel/metadata"
"github.com/opentracing/opentracing-go"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
@ -209,7 +210,9 @@ func (q *query) resultLength(res promql_parser.Value) int {
// Exec Implements `Query`. It handles instrumentation & defers to Eval.
func (q *query) Exec(ctx context.Context) (logqlmodel.Result, error) {
log, ctx := spanlogger.New(ctx, "query.Exec")
sp, ctx := opentracing.StartSpanFromContext(ctx, "query.Exec")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
if q.logExecQuery {

@ -10,6 +10,7 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/gorilla/websocket"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql/parser"
@ -487,7 +488,11 @@ func (q *QuerierAPI) validateMaxEntriesLimits(ctx context.Context, query string,
func WrapQuerySpanAndTimeout(call string, q *QuerierAPI) middleware.Interface {
return middleware.Func(func(next http.Handler) http.Handler {
return http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) {
log, ctx := spanlogger.New(req.Context(), call)
sp, ctx := opentracing.StartSpanFromContext(req.Context(), call)
defer sp.Finish()
log := spanlogger.FromContext(req.Context())
defer log.Finish()
tenants, err := tenant.TenantIDs(ctx)
if err != nil {
level.Error(log).Log("msg", "couldn't fetch tenantID", "err", err)

@ -3,9 +3,11 @@ package queryrange
import (
"context"
"fmt"
reflect "reflect"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/tenant"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
"github.com/prometheus/prometheus/promql/parser"
@ -90,9 +92,11 @@ type instance struct {
func (in instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery) ([]logqlmodel.Result, error) {
return in.For(ctx, queries, func(qry logql.DownstreamQuery) (logqlmodel.Result, error) {
req := ParamsToLokiRequest(qry.Params, qry.Shards).WithQuery(qry.Expr.String())
logger, ctx := spanlogger.New(ctx, "DownstreamHandler.instance")
sp, ctx := opentracing.StartSpanFromContext(ctx, "DownstreamHandler.instance")
defer sp.Finish()
logger := spanlogger.FromContext(ctx)
defer logger.Finish()
level.Debug(logger).Log("shards", fmt.Sprintf("%+v", qry.Shards), "query", req.GetQuery(), "step", req.GetStep())
level.Debug(logger).Log("shards", fmt.Sprintf("%+v", qry.Shards), "query", req.GetQuery(), "step", req.GetStep(), "handler", reflect.TypeOf(in.handler))
res, err := in.handler.Do(ctx, req)
if err != nil {

@ -150,7 +150,9 @@ func NewLimitsMiddleware(l Limits) queryrangebase.Middleware {
}
func (l limitsMiddleware) Do(ctx context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
log, ctx := spanlogger.New(ctx, "limits")
span, ctx := opentracing.StartSpanFromContext(ctx, "limits")
defer span.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
tenantIDs, err := tenant.TenantIDs(ctx)
@ -278,8 +280,10 @@ func NewQuerySizeLimiterMiddleware(
// - {job="foo"}
// - {job="bar"}
func (q *querySizeLimiter) getBytesReadForRequest(ctx context.Context, r queryrangebase.Request) (uint64, error) {
sp, ctx := spanlogger.NewWithLogger(ctx, q.logger, "querySizeLimiter.getBytesReadForRequest")
sp, ctx := opentracing.StartSpanFromContext(ctx, "querySizeLimiter.getBytesReadForRequest")
defer sp.Finish()
log := spanlogger.FromContextWithFallback(ctx, q.logger)
defer log.Finish()
expr, err := syntax.ParseExpr(r.GetQuery())
if err != nil {
@ -301,7 +305,7 @@ func (q *querySizeLimiter) getBytesReadForRequest(ctx context.Context, r queryra
combinedStats := stats.MergeStats(matcherStats...)
level.Debug(sp).Log(
level.Debug(log).Log(
append(
combinedStats.LoggingKeyValues(),
"msg", "queried index",
@ -341,7 +345,9 @@ func (q *querySizeLimiter) guessLimitName() string {
}
func (q *querySizeLimiter) Do(ctx context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
log, ctx := spanlogger.New(ctx, "query_size_limits")
span, ctx := opentracing.StartSpanFromContext(ctx, "query_size_limits")
defer span.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
// Only support TSDB
@ -558,6 +564,9 @@ func (rt limitedRoundTripper) RoundTrip(r *http.Request) (*http.Response, error)
}
func (rt limitedRoundTripper) do(ctx context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "limitedRoundTripper.do")
defer sp.Finish()
request, err := rt.codec.EncodeRequest(ctx, r)
if err != nil {
return nil, err

@ -9,6 +9,7 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/gogo/protobuf/proto"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
@ -80,6 +81,8 @@ type logResultCache struct {
}
func (l *logResultCache) Do(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "logResultCache.Do")
defer sp.Finish()
tenantIDs, err := tenant.TenantIDs(ctx)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())

@ -231,7 +231,9 @@ func (prometheusCodec) DecodeResponse(ctx context.Context, r *http.Response, _ R
body, _ := io.ReadAll(r.Body)
return nil, httpgrpc.Errorf(r.StatusCode, string(body))
}
log, ctx := spanlogger.New(ctx, "ParseQueryRangeResponse") //nolint:ineffassign,staticcheck
sp, ctx := opentracing.StartSpanFromContext(ctx, "ParseQueryRangeResponse") //nolint:ineffassign,staticcheck
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
buf, err := bodyBuffer(r)

@ -212,6 +212,8 @@ func NewResultsCacheMiddleware(
}
func (s resultsCache) Do(ctx context.Context, r Request) (Response, error) {
sp, ctx := opentracing.StartSpanFromContext(ctx, "resultsCache.Do")
defer sp.Finish()
tenantIDs, err := tenant.TenantIDs(ctx)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
@ -394,7 +396,9 @@ func (s resultsCache) handleHit(ctx context.Context, r Request, extents []Extent
reqResps []RequestResponse
err error
)
log, ctx := spanlogger.New(ctx, "handleHit")
sp, ctx := opentracing.StartSpanFromContext(ctx, "handleHit")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
requests, responses, err := s.partition(r, extents)
@ -607,7 +611,9 @@ func (s resultsCache) get(ctx context.Context, key string) ([]Extent, bool) {
}
var resp CachedResponse
log, ctx := spanlogger.New(ctx, "unmarshal-extent") //nolint:ineffassign,staticcheck
sp, ctx := opentracing.StartSpanFromContext(ctx, "unmarshal-extent") //nolint:ineffassign,staticcheck
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
log.LogFields(otlog.Int("bytes", len(bufs[0])))

@ -18,6 +18,7 @@ import (
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/index/stats"
"github.com/grafana/loki/pkg/util/spanlogger"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
)
@ -128,8 +129,10 @@ func getStatsForMatchers(
}
func (r *dynamicShardResolver) GetStats(e syntax.Expr) (stats.Stats, error) {
sp, ctx := spanlogger.NewWithLogger(r.ctx, r.logger, "dynamicShardResolver.GetStats")
sp, ctx := opentracing.StartSpanFromContext(r.ctx, "dynamicShardResolver.GetStats")
defer sp.Finish()
log := spanlogger.FromContext(r.ctx)
defer log.Finish()
start := time.Now()
@ -146,14 +149,14 @@ func (r *dynamicShardResolver) GetStats(e syntax.Expr) (stats.Stats, error) {
grps = append(grps, syntax.MatcherRange{})
}
results, err := getStatsForMatchers(ctx, sp, r.handler, r.from, r.through, grps, r.maxParallelism, r.defaultLookback)
results, err := getStatsForMatchers(ctx, log, r.handler, r.from, r.through, grps, r.maxParallelism, r.defaultLookback)
if err != nil {
return stats.Stats{}, err
}
combined := stats.MergeStats(results...)
level.Debug(sp).Log(
level.Debug(log).Log(
append(
combined.LoggingKeyValues(),
"msg", "queried index",
@ -168,8 +171,10 @@ func (r *dynamicShardResolver) GetStats(e syntax.Expr) (stats.Stats, error) {
}
func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) {
sp, _ := spanlogger.NewWithLogger(r.ctx, r.logger, "dynamicShardResolver.Shards")
sp, ctx := opentracing.StartSpanFromContext(r.ctx, "dynamicShardResolver.Shards")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
combined, err := r.GetStats(e)
if err != nil {
@ -183,7 +188,7 @@ func (r *dynamicShardResolver) Shards(e syntax.Expr) (int, uint64, error) {
bytesPerShard = combined.Bytes / uint64(factor)
}
level.Debug(sp).Log(
level.Debug(log).Log(
append(
combined.LoggingKeyValues(),
"msg", "Got shard factor",

@ -4,6 +4,7 @@ import (
"context"
"sync"
"github.com/opentracing/opentracing-go"
otlog "github.com/opentracing/opentracing-go/log"
"github.com/grafana/loki/pkg/storage/chunk"
@ -18,7 +19,9 @@ var decodeContextPool = sync.Pool{
// GetParallelChunks fetches chunks in parallel (up to maxParallel).
func GetParallelChunks(ctx context.Context, maxParallel int, chunks []chunk.Chunk, f func(context.Context, *chunk.DecodeContext, chunk.Chunk) (chunk.Chunk, error)) ([]chunk.Chunk, error) {
log, ctx := spanlogger.New(ctx, "GetParallelChunks")
sp, ctx := opentracing.StartSpanFromContext(ctx, "GetParallelChunks")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
log.LogFields(otlog.Int("requested", len(chunks)))

@ -6,6 +6,7 @@ import (
"sync"
"github.com/go-kit/log/level"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/prometheus/promql"
@ -174,7 +175,9 @@ func (c *Fetcher) FetchChunks(ctx context.Context, chunks []chunk.Chunk, keys []
if ctx.Err() != nil {
return nil, ctx.Err()
}
log, ctx := spanlogger.New(ctx, "ChunkStore.FetchChunks")
sp, ctx := opentracing.StartSpanFromContext(ctx, "ChunkStore.FetchChunks")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
// Now fetch the actual chunk data from Memcache / S3

@ -6,6 +6,7 @@ import (
"time"
"github.com/go-kit/log/level"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
@ -48,7 +49,9 @@ func (c *storeEntry) GetChunkRefs(ctx context.Context, userID string, from, thro
if ctx.Err() != nil {
return nil, nil, ctx.Err()
}
log, ctx := spanlogger.New(ctx, "GetChunkRefs")
sp, ctx := opentracing.StartSpanFromContext(ctx, "GetChunkRefs")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through)
@ -86,7 +89,9 @@ func (c *storeEntry) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) {
// LabelNamesForMetricName retrieves all label names for a metric name.
func (c *storeEntry) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string) ([]string, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.LabelNamesForMetricName")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelNamesForMetricName")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through)
@ -101,7 +106,9 @@ func (c *storeEntry) LabelNamesForMetricName(ctx context.Context, userID string,
}
func (c *storeEntry) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.LabelValuesForMetricName")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelValuesForMetricName")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through)
@ -115,8 +122,8 @@ func (c *storeEntry) LabelValuesForMetricName(ctx context.Context, userID string
}
func (c *storeEntry) Stats(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) (*stats.Stats, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.Stats")
defer log.Span.Finish()
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.Stats")
defer sp.Finish()
shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through)
if err != nil {

@ -8,6 +8,7 @@ import (
"github.com/go-kit/log/level"
jsoniter "github.com/json-iterator/go"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
@ -276,7 +277,9 @@ func (c *indexReaderWriter) chunksToSeries(ctx context.Context, in []logproto.Ch
// LabelNamesForMetricName retrieves all label names for a metric name.
func (c *indexReaderWriter) LabelNamesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string) ([]string, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.LabelNamesForMetricName")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelNamesForMetricName")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
// Fetch the series IDs from the index
@ -302,7 +305,9 @@ func (c *indexReaderWriter) LabelNamesForMetricName(ctx context.Context, userID
}
func (c *indexReaderWriter) LabelValuesForMetricName(ctx context.Context, userID string, from, through model.Time, metricName string, labelName string, matchers ...*labels.Matcher) ([]string, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.LabelValuesForMetricName")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.LabelValuesForMetricName")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
if len(matchers) != 0 {
@ -587,7 +592,9 @@ func (c *indexReaderWriter) lookupEntriesByQueries(ctx context.Context, queries
}
func (c *indexReaderWriter) lookupLabelNamesBySeries(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.lookupLabelNamesBySeries")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.lookupLabelNamesBySeries")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
level.Debug(log).Log("seriesIDs", len(seriesIDs))
@ -622,7 +629,9 @@ func (c *indexReaderWriter) lookupLabelNamesBySeries(ctx context.Context, from,
}
func (c *indexReaderWriter) lookupLabelNamesByChunks(ctx context.Context, from, through model.Time, userID string, seriesIDs []string) ([]string, error) {
log, ctx := spanlogger.New(ctx, "SeriesStore.lookupLabelNamesByChunks")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.lookupLabelNamesByChunks")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Span.Finish()
// Lookup the series in the index to get the chunks.

@ -4,6 +4,7 @@ import (
"context"
"github.com/go-kit/log/level"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
@ -59,7 +60,9 @@ func (c *Writer) Put(ctx context.Context, chunks []chunk.Chunk) error {
// PutOne implements Store
func (c *Writer) PutOne(ctx context.Context, from, through model.Time, chk chunk.Chunk) error {
log, ctx := spanlogger.New(ctx, "SeriesStore.PutOne")
sp, ctx := opentracing.StartSpanFromContext(ctx, "SeriesStore.PutOne")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
var (

@ -4,6 +4,7 @@ import (
"context"
"time"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
@ -92,8 +93,10 @@ func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, *index.Shard
// They share almost the same fields, so we can add the missing `KB` field to the proto and then
// use that within the tsdb package.
func (c *IndexClient) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]logproto.ChunkRef, error) {
log, ctx := spanlogger.New(ctx, "IndexClient.GetChunkRefs")
defer log.Span.Finish()
sp, ctx := opentracing.StartSpanFromContext(ctx, "IndexClient.GetChunkRefs")
defer sp.Finish()
log := spanlogger.FromContext(ctx)
defer log.Finish()
var kvps []interface{}
defer func() {

Loading…
Cancel
Save