diff --git a/CHANGELOG.md b/CHANGELOG.md index 0afa1013b51..13664f220a5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -62,6 +62,9 @@ * [ENHANCEMENT] Reduce ingester working set by improving prelloc behavior. [#4344](https://github.com/grafana/tempo/pull/4344) (@joe-elliott) * [ENHANCEMENT] Use Promtheus fast regexp for TraceQL regular expression matchers. [#4329](https://github.com/grafana/tempo/pull/4329) (@joe-elliott) **BREAKING CHANGE** All regular expression matchers will now be fully anchored. `span.foo =~ "bar"` will now be evaluated as `span.foo =~ "^bar$"` +* [ENHANCEMENT] Reuse generator code to better refuse "too large" traces. [#4365](https://github.com/grafana/tempo/pull/4365) (@joe-elliott) + This will cause the ingester to more aggressively and correctly refuse traces. Also added two metrics to better track bytes consumed per tenant in the ingester. + `tempo_metrics_generator_live_trace_bytes` and `tempo_ingester_live_trace_bytes`. * [BUGFIX] Replace hedged requests roundtrips total with a counter. [#4063](https://github.com/grafana/tempo/pull/4063) [#4078](https://github.com/grafana/tempo/pull/4078) (@galalen) * [BUGFIX] Metrics generators: Correctly drop from the ring before stopping ingestion to reduce drops during a rollout. [#4101](https://github.com/grafana/tempo/pull/4101) (@joe-elliott) * [BUGFIX] Correctly handle 400 Bad Request and 404 Not Found in gRPC streaming [#4144](https://github.com/grafana/tempo/pull/4144) (@mapno) @@ -71,9 +74,9 @@ * [BUGFIX] Fix several issues with exemplar values for traceql metrics [#4366](https://github.com/grafana/tempo/pull/4366) (@mdisibio) * [BUGFIX] Skip computing exemplars for instant queries. [#4204](https://github.com/grafana/tempo/pull/4204) (@javiermolinar) * [BUGFIX] Gave context to orphaned spans related to various maintenance processes. [#4260](https://github.com/grafana/tempo/pull/4260) (@joe-elliott) -* [BUGFIX] Utilize S3Pass and S3User parameters in tempo-cli options, which were previously unused in the code. [#44236](https://github.com/grafana/tempo/pull/4259) (@faridtmammadov) * [BUGFIX] Initialize histogram buckets to 0 to avoid downsampling. [#4366](https://github.com/grafana/tempo/pull/4366) (@javiermolinar) - +* [BUGFIX] Utilize S3Pass and S3User parameters in tempo-cli options, which were previously unused in the code. [#4259](https://github.com/grafana/tempo/pull/4259) (@faridtmammadov) +* [BUGFIX] Fixed an issue in the generator where the first batch was counted 2x against a traces size. [#4365](https://github.com/grafana/tempo/pull/4365) (@joe-elliott) # v2.6.1 diff --git a/integration/e2e/limits_test.go b/integration/e2e/limits_test.go index 2540214ccde..cc02ca80c22 100644 --- a/integration/e2e/limits_test.go +++ b/integration/e2e/limits_test.go @@ -221,6 +221,9 @@ func TestQueryLimits(t *testing.T) { batch.Spans = allSpans[i : i+1] require.NoError(t, c.EmitBatch(context.Background(), batch)) util.CallFlush(t, tempo) + // this push along with the double flush is required to forget the too large trace + require.NoError(t, c.EmitBatch(context.Background(), util.MakeThriftBatchWithSpanCount(1))) + util.CallFlush(t, tempo) time.Sleep(2 * time.Second) // trace idle and flush time are both 1ms } diff --git a/modules/generator/processor/localblocks/livetraces.go b/modules/generator/processor/localblocks/livetraces.go index 468025758db..9d0ec8231b1 100644 --- a/modules/generator/processor/localblocks/livetraces.go +++ b/modules/generator/processor/localblocks/livetraces.go @@ -12,11 +12,15 @@ type liveTrace struct { id []byte timestamp time.Time Batches []*v1.ResourceSpans + + sz uint64 } type liveTraces struct { hash hash.Hash64 traces map[uint64]*liveTrace + + sz uint64 } func newLiveTraces() *liveTraces { @@ -36,6 +40,10 @@ func (l *liveTraces) Len() uint64 { return uint64(len(l.traces)) } +func (l *liveTraces) Size() uint64 { + return l.sz +} + func (l *liveTraces) Push(traceID []byte, batch *v1.ResourceSpans, max uint64) bool { token := l.token(traceID) @@ -54,6 +62,10 @@ func (l *liveTraces) Push(traceID []byte, batch *v1.ResourceSpans, max uint64) b l.traces[token] = tr } + sz := uint64(batch.Size()) + tr.sz += sz + l.sz += sz + tr.Batches = append(tr.Batches, batch) tr.timestamp = time.Now() return true @@ -65,6 +77,7 @@ func (l *liveTraces) CutIdle(idleSince time.Time, immediate bool) []*liveTrace { for k, tr := range l.traces { if tr.timestamp.Before(idleSince) || immediate { res = append(res, tr) + l.sz -= tr.sz delete(l.traces, k) } } diff --git a/modules/generator/processor/localblocks/livetraces_test.go b/modules/generator/processor/localblocks/livetraces_test.go new file mode 100644 index 00000000000..60a1f3be12a --- /dev/null +++ b/modules/generator/processor/localblocks/livetraces_test.go @@ -0,0 +1,46 @@ +package localblocks + +import ( + "math/rand/v2" + "testing" + "time" + + "github.com/grafana/tempo/pkg/util/test" + "github.com/stretchr/testify/require" +) + +func TestLiveTracesSizesAndLen(t *testing.T) { + lt := newLiveTraces() + + expectedSz := uint64(0) + expectedLen := uint64(0) + + for i := 0; i < 100; i++ { + id := test.ValidTraceID(nil) + tr := test.MakeTrace(rand.IntN(5)+1, id) + + cutTime := time.Now() + + // add some traces and confirm size/len + expectedLen++ + for _, rs := range tr.ResourceSpans { + expectedSz += uint64(rs.Size()) + lt.Push(id, rs, 0) + } + + require.Equal(t, expectedSz, lt.Size()) + require.Equal(t, expectedLen, lt.Len()) + + // cut some traces and confirm size/len + cutTraces := lt.CutIdle(cutTime, false) + for _, tr := range cutTraces { + for _, rs := range tr.Batches { + expectedSz -= uint64(rs.Size()) + } + expectedLen-- + } + + require.Equal(t, expectedSz, lt.Size()) + require.Equal(t, expectedLen, lt.Len()) + } +} diff --git a/modules/generator/processor/localblocks/metrics.go b/modules/generator/processor/localblocks/metrics.go index aaa8bbe41f9..5e580ca5e04 100644 --- a/modules/generator/processor/localblocks/metrics.go +++ b/modules/generator/processor/localblocks/metrics.go @@ -38,6 +38,12 @@ var ( Name: "live_traces", Help: "Number of live traces", }, []string{"tenant"}) + metricLiveTraceBytes = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "live_trace_bytes", + Help: "Total number of traces created", + }, []string{"tenant"}) metricDroppedTraces = promauto.NewCounterVec(prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, diff --git a/modules/generator/processor/localblocks/processor.go b/modules/generator/processor/localblocks/processor.go index ec3458eda43..78dca7bdc4a 100644 --- a/modules/generator/processor/localblocks/processor.go +++ b/modules/generator/processor/localblocks/processor.go @@ -16,6 +16,7 @@ import ( "github.com/google/uuid" "github.com/grafana/tempo/modules/ingester" "github.com/grafana/tempo/pkg/flushqueues" + "github.com/grafana/tempo/pkg/tracesizes" "github.com/grafana/tempo/tempodb" "go.opentelemetry.io/otel" @@ -70,7 +71,7 @@ type Processor struct { liveTracesMtx sync.Mutex liveTraces *liveTraces - traceSizes *traceSizes + traceSizes *tracesizes.Tracker writer tempodb.Writer } @@ -103,7 +104,7 @@ func New(cfg Config, tenant string, wal *wal.WAL, writer tempodb.Writer, overrid completeBlocks: map[uuid.UUID]*ingester.LocalBlock{}, flushqueue: flushqueues.NewPriorityQueue(metricFlushQueueSize.WithLabelValues(tenant)), liveTraces: newLiveTraces(), - traceSizes: newTraceSizes(), + traceSizes: tracesizes.New(), closeCh: make(chan struct{}), wg: sync.WaitGroup{}, cache: lru.New(100), @@ -597,6 +598,7 @@ func (p *Processor) cutIdleTraces(immediate bool) error { // Record live traces before flushing so we know the high water mark metricLiveTraces.WithLabelValues(p.tenant).Set(float64(len(p.liveTraces.traces))) + metricLiveTraceBytes.WithLabelValues(p.tenant).Set(float64(p.liveTraces.Size())) since := time.Now().Add(-p.Cfg.TraceIdlePeriod) tracesToCut := p.liveTraces.CutIdle(since, immediate) diff --git a/modules/ingester/instance.go b/modules/ingester/instance.go index 26b4b062567..b1ce94d8b4f 100644 --- a/modules/ingester/instance.go +++ b/modules/ingester/instance.go @@ -17,13 +17,13 @@ import ( "github.com/google/uuid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "go.uber.org/atomic" "google.golang.org/grpc/codes" "github.com/grafana/tempo/modules/overrides" "github.com/grafana/tempo/pkg/model" "github.com/grafana/tempo/pkg/model/trace" "github.com/grafana/tempo/pkg/tempopb" + "github.com/grafana/tempo/pkg/tracesizes" "github.com/grafana/tempo/pkg/util/log" "github.com/grafana/tempo/pkg/validation" "github.com/grafana/tempo/tempodb" @@ -64,6 +64,11 @@ var ( Name: "ingester_live_traces", Help: "The current number of lives traces per tenant.", }, []string{"tenant"}) + metricLiveTraceBytes = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "tempo", + Name: "ingester_live_trace_bytes", + Help: "The current number of bytes consumed by lives traces per tenant.", + }, []string{"tenant"}) metricBlocksClearedTotal = promauto.NewCounter(prometheus.CounterOpts{ Namespace: "tempo", Name: "ingester_blocks_cleared_total", @@ -82,10 +87,10 @@ var ( ) type instance struct { - tracesMtx sync.Mutex - traces map[uint32]*liveTrace - traceSizes map[uint32]uint32 - traceCount atomic.Int32 + tracesMtx sync.Mutex + traces map[uint32]*liveTrace + traceSizes *tracesizes.Tracker + traceSizeBytes uint64 headBlockMtx sync.RWMutex headBlock common.WALBlock @@ -115,7 +120,7 @@ type instance struct { func newInstance(instanceID string, limiter *Limiter, overrides ingesterOverrides, writer tempodb.Writer, l *local.Backend, dedicatedColumns backend.DedicatedColumns) (*instance, error) { i := &instance{ traces: map[uint32]*liveTrace{}, - traceSizes: map[uint32]uint32{}, + traceSizes: tracesizes.New(), instanceID: instanceID, tracesCreatedTotal: metricTracesCreatedTotal.WithLabelValues(instanceID), @@ -190,12 +195,6 @@ func (i *instance) PushBytes(ctx context.Context, id, traceBytes []byte) error { return status.Errorf(codes.InvalidArgument, "%s is not a valid traceid", hex.EncodeToString(id)) } - // check for max traces before grabbing the lock to better load shed - err := i.limiter.AssertMaxTracesPerUser(i.instanceID, int(i.traceCount.Load())) - if err != nil { - return newMaxLiveTracesError(i.instanceID, err.Error()) - } - return i.push(ctx, id, traceBytes) } @@ -203,27 +202,27 @@ func (i *instance) push(ctx context.Context, id, traceBytes []byte) error { i.tracesMtx.Lock() defer i.tracesMtx.Unlock() - tkn := i.tokenForTraceID(id) + err := i.limiter.AssertMaxTracesPerUser(i.instanceID, len(i.traces)) + if err != nil { + return newMaxLiveTracesError(i.instanceID, err.Error()) + } + maxBytes := i.limiter.limits.MaxBytesPerTrace(i.instanceID) + reqSize := len(traceBytes) - if maxBytes > 0 { - prevSize := int(i.traceSizes[tkn]) - reqSize := len(traceBytes) - if prevSize+reqSize > maxBytes { - return newTraceTooLargeError(id, i.instanceID, maxBytes, reqSize) - } + if maxBytes > 0 && !i.traceSizes.Allow(id, reqSize, maxBytes) { + return newTraceTooLargeError(id, i.instanceID, maxBytes, reqSize) } - trace := i.getOrCreateTrace(id, tkn, maxBytes) + tkn := i.tokenForTraceID(id) + trace := i.getOrCreateTrace(id, tkn) - err := trace.Push(ctx, i.instanceID, traceBytes) + err = trace.Push(ctx, i.instanceID, traceBytes) if err != nil { return err } - if maxBytes > 0 { - i.traceSizes[tkn] += uint32(len(traceBytes)) - } + i.traceSizeBytes += uint64(reqSize) return nil } @@ -281,6 +280,8 @@ func (i *instance) CutBlockIfReady(maxBlockLifetime time.Duration, maxBlockBytes now := time.Now() if i.lastBlockCut.Add(maxBlockLifetime).Before(now) || i.headBlock.DataLength() >= maxBlockBytes || immediate { + // Reset trace sizes when cutting block + i.traceSizes.ClearIdle(i.lastBlockCut) // Final flush err := i.headBlock.Flush() @@ -485,15 +486,14 @@ func (i *instance) AddCompletingBlock(b common.WALBlock) { // getOrCreateTrace will return a new trace object for the given request // // It must be called under the i.tracesMtx lock -func (i *instance) getOrCreateTrace(traceID []byte, fp uint32, maxBytes int) *liveTrace { +func (i *instance) getOrCreateTrace(traceID []byte, fp uint32) *liveTrace { trace, ok := i.traces[fp] if ok { return trace } - trace = newTrace(traceID, maxBytes) + trace = newTrace(traceID) i.traces[fp] = trace - i.traceCount.Inc() return trace } @@ -507,11 +507,6 @@ func (i *instance) tokenForTraceID(id []byte) uint32 { // resetHeadBlock() should be called under lock func (i *instance) resetHeadBlock() error { - // Reset trace sizes when cutting block - i.tracesMtx.Lock() - i.traceSizes = make(map[uint32]uint32, len(i.traceSizes)) - i.tracesMtx.Unlock() - dedicatedColumns := i.getDedicatedColumns() meta := &backend.BlockMeta{ @@ -549,6 +544,7 @@ func (i *instance) tracesToCut(cutoff time.Duration, immediate bool) []*liveTrac // Set this before cutting to give a more accurate number. metricLiveTraces.WithLabelValues(i.instanceID).Set(float64(len(i.traces))) + metricLiveTraceBytes.WithLabelValues(i.instanceID).Set(float64(i.traceSizeBytes)) cutoffTime := time.Now().Add(cutoff) tracesToCut := make([]*liveTrace, 0, len(i.traces)) @@ -556,10 +552,13 @@ func (i *instance) tracesToCut(cutoff time.Duration, immediate bool) []*liveTrac for key, trace := range i.traces { if cutoffTime.After(trace.lastAppend) || immediate { tracesToCut = append(tracesToCut, trace) + + // decrease live trace bytes + i.traceSizeBytes -= trace.Size() + delete(i.traces, key) } } - i.traceCount.Store(int32(len(i.traces))) return tracesToCut } diff --git a/modules/ingester/instance_search_test.go b/modules/ingester/instance_search_test.go index 84a20a920d2..a3be2669e1e 100644 --- a/modules/ingester/instance_search_test.go +++ b/modules/ingester/instance_search_test.go @@ -115,7 +115,6 @@ func TestInstanceSearchTraceQL(t *testing.T) { // Test after appending to WAL require.NoError(t, i.CutCompleteTraces(0, true)) - assert.Equal(t, int(i.traceCount.Load()), len(i.traces)) sr, err = i.Search(context.Background(), req) assert.NoError(t, err) @@ -597,8 +596,6 @@ func writeTracesForSearch(t *testing.T, i *instance, spanName, tagKey, tagValue // searchData will be nil if not err = i.PushBytes(context.Background(), id, traceBytes) require.NoError(t, err) - - assert.Equal(t, int(i.traceCount.Load()), len(i.traces)) } // traces have to be cut to show up in searches @@ -805,8 +802,6 @@ func TestInstanceSearchMetrics(t *testing.T) { err = i.PushBytes(context.Background(), id, traceBytes) require.NoError(t, err) - - assert.Equal(t, int(i.traceCount.Load()), len(i.traces)) } search := func() *tempopb.SearchMetrics { diff --git a/modules/ingester/instance_test.go b/modules/ingester/instance_test.go index 436a086ae55..fe1d1bc956f 100644 --- a/modules/ingester/instance_test.go +++ b/modules/ingester/instance_test.go @@ -35,16 +35,20 @@ func (m *ringCountMock) HealthyInstancesCount() int { func TestInstance(t *testing.T) { request := makeRequest([]byte{}) + requestSz := uint64(0) + for _, b := range request.Traces { + requestSz += uint64(len(b.Slice)) + } i, ingester := defaultInstance(t) response := i.PushBytesRequest(context.Background(), request) require.NotNil(t, response) - require.Equal(t, int(i.traceCount.Load()), len(i.traces)) + require.Equal(t, requestSz, i.traceSizeBytes) err := i.CutCompleteTraces(0, true) require.NoError(t, err) - require.Equal(t, int(i.traceCount.Load()), len(i.traces)) + require.Equal(t, uint64(0), i.traceSizeBytes) blockID, err := i.CutBlockIfReady(0, 0, false) require.NoError(t, err, "unexpected error cutting block") @@ -71,8 +75,6 @@ func TestInstance(t *testing.T) { err = i.resetHeadBlock() require.NoError(t, err, "unexpected error resetting block") - - require.Equal(t, int(i.traceCount.Load()), len(i.traces)) } func TestInstanceFind(t *testing.T) { @@ -85,7 +87,6 @@ func TestInstanceFind(t *testing.T) { err := i.CutCompleteTraces(0, true) require.NoError(t, err) - require.Equal(t, int(i.traceCount.Load()), len(i.traces)) for j := 0; j < numTraces; j++ { traceBytes, err := model.MustNewSegmentDecoder(model.CurrentEncoding).PrepareForWrite(traces[j], 0, 0) @@ -140,7 +141,6 @@ func pushTracesToInstance(t *testing.T, i *instance, numTraces int) ([]*tempopb. err = i.PushBytes(context.Background(), id, traceBytes) require.NoError(t, err) - require.Equal(t, int(i.traceCount.Load()), len(i.traces)) ids = append(ids, id) traces = append(traces, testTrace) @@ -570,9 +570,26 @@ func TestInstanceFailsLargeTracesEvenAfterFlushing(t *testing.T) { _, _, traceTooLargeCount = CheckPushBytesError(response) assert.Equal(t, true, traceTooLargeCount > 0) - // Cut block and then pushing works again + // Cut block and then pushing still fails b/c too large traces persist til they stop being pushed _, err = i.CutBlockIfReady(0, 0, true) require.NoError(t, err) + response = i.PushBytesRequest(ctx, req) + _, _, traceTooLargeCount = CheckPushBytesError(response) + assert.Equal(t, true, traceTooLargeCount > 0) + + // Cut block 2x w/o while pushing other traces, but not the problematic trace! this will finally clear the trace + i.PushBytesRequest(ctx, makeRequestWithByteLimit(200, nil)) + err = i.CutCompleteTraces(0, true) + require.NoError(t, err) + _, err = i.CutBlockIfReady(0, 0, true) + require.NoError(t, err) + + i.PushBytesRequest(ctx, makeRequestWithByteLimit(200, nil)) + err = i.CutCompleteTraces(0, true) + require.NoError(t, err) + _, err = i.CutBlockIfReady(0, 0, true) + require.NoError(t, err) + response = i.PushBytesRequest(ctx, req) errored, _, _ = CheckPushBytesError(response) require.False(t, errored, "push failed: %w", response.ErrorsByTrace) @@ -813,7 +830,9 @@ func makeRequestWithByteLimit(maxBytes int, traceID []byte) *tempopb.PushBytesRe traceID = test.ValidTraceID(traceID) batch := makeBatchWithMaxBytes(maxBytes, traceID) - return makePushBytesRequest(traceID, batch) + pushReq := makePushBytesRequest(traceID, batch) + + return pushReq } func makePushBytesRequest(traceID []byte, batch *v1_trace.ResourceSpans) *tempopb.PushBytesRequest { diff --git a/modules/ingester/trace.go b/modules/ingester/trace.go index c0f5c0ba26e..60fee397a53 100644 --- a/modules/ingester/trace.go +++ b/modules/ingester/trace.go @@ -15,32 +15,19 @@ type liveTrace struct { start uint32 end uint32 decoder model.SegmentDecoder - - // byte limits - maxBytes int - currentBytes int } -func newTrace(traceID []byte, maxBytes int) *liveTrace { +func newTrace(traceID []byte) *liveTrace { return &liveTrace{ batches: make([][]byte, 0, 10), // 10 for luck lastAppend: time.Now(), traceID: traceID, - maxBytes: maxBytes, decoder: model.MustNewSegmentDecoder(model.CurrentEncoding), } } func (t *liveTrace) Push(_ context.Context, instanceID string, trace []byte) error { t.lastAppend = time.Now() - if t.maxBytes != 0 { - reqSize := len(trace) - if t.currentBytes+reqSize > t.maxBytes { - return newTraceTooLargeError(t.traceID, instanceID, t.maxBytes, reqSize) - } - - t.currentBytes += reqSize - } start, end, err := t.decoder.FastRange(trace) if err != nil { @@ -56,3 +43,11 @@ func (t *liveTrace) Push(_ context.Context, instanceID string, trace []byte) err return nil } + +func (t *liveTrace) Size() uint64 { + size := uint64(0) + for _, batch := range t.batches { + size += uint64(len(batch)) + } + return size +} diff --git a/modules/ingester/trace_test.go b/modules/ingester/trace_test.go index ce6e251849d..f276ccbf4b7 100644 --- a/modules/ingester/trace_test.go +++ b/modules/ingester/trace_test.go @@ -13,7 +13,7 @@ import ( func TestTraceStartEndTime(t *testing.T) { s := model.MustNewSegmentDecoder(model.CurrentEncoding) - tr := newTrace(nil, 0) + tr := newTrace(nil) // initial push buff, err := s.PrepareForWrite(&tempopb.Trace{}, 10, 20) diff --git a/modules/generator/processor/localblocks/traceSizes.go b/pkg/tracesizes/traceSizes.go similarity index 76% rename from modules/generator/processor/localblocks/traceSizes.go rename to pkg/tracesizes/traceSizes.go index 54cfa639c97..730213d1d9d 100644 --- a/modules/generator/processor/localblocks/traceSizes.go +++ b/pkg/tracesizes/traceSizes.go @@ -1,4 +1,4 @@ -package localblocks +package tracesizes import ( "hash" @@ -7,7 +7,7 @@ import ( "time" ) -type traceSizes struct { +type Tracker struct { mtx sync.Mutex hash hash.Hash64 sizes map[uint64]*traceSize @@ -18,14 +18,14 @@ type traceSize struct { timestamp time.Time } -func newTraceSizes() *traceSizes { - return &traceSizes{ +func New() *Tracker { + return &Tracker{ hash: fnv.New64(), sizes: make(map[uint64]*traceSize), } } -func (s *traceSizes) token(traceID []byte) uint64 { +func (s *Tracker) token(traceID []byte) uint64 { s.hash.Reset() s.hash.Write(traceID) return s.hash.Sum64() @@ -34,7 +34,7 @@ func (s *traceSizes) token(traceID []byte) uint64 { // Allow returns true if the historical total plus incoming size is less than // or equal to the max. The historical total is kept alive and incremented even // if not allowed, so that long-running traces are cutoff as expected. -func (s *traceSizes) Allow(traceID []byte, sz, max int) bool { +func (s *Tracker) Allow(traceID []byte, sz, max int) bool { s.mtx.Lock() defer s.mtx.Unlock() @@ -42,7 +42,7 @@ func (s *traceSizes) Allow(traceID []byte, sz, max int) bool { tr := s.sizes[token] if tr == nil { tr = &traceSize{ - size: sz, + size: 0, // size added below } s.sizes[token] = tr } @@ -53,7 +53,7 @@ func (s *traceSizes) Allow(traceID []byte, sz, max int) bool { return tr.size <= max } -func (s *traceSizes) ClearIdle(idleSince time.Time) { +func (s *Tracker) ClearIdle(idleSince time.Time) { s.mtx.Lock() defer s.mtx.Unlock()