Skip to content

Commit 52d5d8e

Browse files
craig[bot]alyshanjahani-crl
andcommitted
Merge #158924
158924: tracing: introduce cluster settings for span/trace structured byte limits r=alyshanjahani-crl a=alyshanjahani-crl This commit introduces two cluster settings that allow us to configure the previous constants maxStructuredBytesPerTrace, and maxStructuredBytesPerSpan. Fixes: #87539 Release note: None Co-authored-by: alyshanjahani-crl <[email protected]>
2 parents 7d65bb4 + 9b8d25b commit 52d5d8e

File tree

4 files changed

+78
-21
lines changed

4 files changed

+78
-21
lines changed

pkg/util/tracing/crdbspan.go

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -758,9 +758,10 @@ func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool, finishing b
758758
result.StructuredRecordsSizeBytes -= result.Root.StructuredRecordsSizeBytes
759759
result.Root = s.getRecordingNoChildrenLocked(tracingpb.RecordingVerbose, finishing)
760760
result.StructuredRecordsSizeBytes += result.Root.StructuredRecordsSizeBytes
761+
maxStructuredBytes := s.tracer.MaxStructuredBytesPerTrace()
761762
for i := range oldEvents {
762763
size := int64(oldEvents[i].Size())
763-
if result.StructuredRecordsSizeBytes+size <= maxStructuredBytesPerTrace {
764+
if result.StructuredRecordsSizeBytes+size <= maxStructuredBytes {
764765
result.Root.AddStructuredRecord(oldEvents[i])
765766
result.StructuredRecordsSizeBytes += size
766767
}
@@ -793,7 +794,7 @@ func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool, finishing b
793794
rollupChildrenMetadata(childrenMetadata, openChildRecording.Root.ChildrenMetadata)
794795
}
795796
}
796-
result.addChildren(openRecordings, maxRecordedSpansPerTrace, maxStructuredBytesPerTrace)
797+
result.addChildren(openRecordings, maxRecordedSpansPerTrace, maxStructuredBytes)
797798
}()
798799

799800
// Copy over the OperationMetadata collected from s' children into the root of
@@ -881,17 +882,18 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRec Trace) {
881882
// processors run in spans that FollowFrom an RPC Span that we don't
882883
// collect.
883884
childRec.Root.ParentSpanID = s.spanID
884-
s.mu.recording.finishedChildren.addChildren([]Trace{childRec}, maxRecordedSpansPerTrace, maxStructuredBytesPerTrace)
885+
s.mu.recording.finishedChildren.addChildren([]Trace{childRec}, maxRecordedSpansPerTrace, s.tracer.MaxStructuredBytesPerTrace())
885886
case tracingpb.RecordingStructured:
886887
fc := &s.mu.recording.finishedChildren
887888
num := len(fc.Root.StructuredRecords)
888889
fc.Root.StructuredRecords = childRec.appendStructuredEventsRecursively(fc.Root.StructuredRecords)
889890
// Account for the size of the structured records that were appended,
890891
// breaking out of the loop if we hit the byte limit. This incorporates
891892
// the byte size accounting logic from RecordedSpan.AddStructuredRecord.
893+
maxStructuredBytes := s.tracer.MaxStructuredBytesPerTrace()
892894
for ; num < len(fc.Root.StructuredRecords); num++ {
893895
size := int64(fc.Root.StructuredRecords[num].MemorySize())
894-
if fc.StructuredRecordsSizeBytes+size > maxStructuredBytesPerTrace {
896+
if fc.StructuredRecordsSizeBytes+size > maxStructuredBytes {
895897
break
896898
}
897899
fc.Root.StructuredRecordsSizeBytes += size

pkg/util/tracing/span.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -716,7 +716,7 @@ func (sp *Span) reset(
716716
goroutineID: goroutineID,
717717
recording: recordingState{
718718
logs: makeSizeLimitedBuffer[*tracingpb.LogRecord](maxLogBytesPerSpan, nil /* scratch */),
719-
structured: makeSizeLimitedBuffer[*tracingpb.StructuredRecord](maxStructuredBytesPerSpan, h.structuredEventsAlloc[:]),
719+
structured: makeSizeLimitedBuffer[*tracingpb.StructuredRecord](sp.i.tracer.MaxStructuredBytesPerSpan(), h.structuredEventsAlloc[:]),
720720
childrenMetadata: h.childrenMetadataAlloc,
721721
finishedChildren: MakeTrace(tracingpb.RecordedSpan{}),
722722
},

pkg/util/tracing/span_test.go

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -316,7 +316,7 @@ func TestSpanRecordStructuredLimit(t *testing.T) {
316316
Payload: anyPayload,
317317
}
318318

319-
numStructuredRecordings := maxStructuredBytesPerSpan / structuredRecord.MemorySize()
319+
numStructuredRecordings := defaultMaxStructuredBytesPerSpan / structuredRecord.MemorySize()
320320
const extra = 10
321321
for i := 1; i <= numStructuredRecordings+extra; i++ {
322322
sp.RecordStructured(payload(i))
@@ -430,7 +430,7 @@ func TestRecordingMaxSpans(t *testing.T) {
430430
}
431431
require.Equal(t, numChildren, numStructuredEvents)
432432
// Same when requesting a Structured recording. Except the exact number of
433-
// events is limited by maxStructuredBytesPerSpan.
433+
// events is limited by defaultMaxStructuredBytesPerSpan.
434434
require.NotEmpty(t, sp.GetRecording(tracingpb.RecordingStructured)[0].StructuredRecords)
435435
}
436436

@@ -760,9 +760,9 @@ func TestRecordingStructuredLogLimit(t *testing.T) {
760760
tr := NewTracer()
761761

762762
// Create one big message. Only one of these fits in a span according to the
763-
// maxStructuredBytesPerSpan limit.
763+
// defaultMaxStructuredBytesPerSpan limit.
764764
var sb strings.Builder
765-
for i := 0; i < maxStructuredBytesPerSpan*7/8; i++ {
765+
for i := 0; i < defaultMaxStructuredBytesPerSpan*7/8; i++ {
766766
sb.WriteRune('c')
767767
}
768768
s := sb.String()
@@ -778,7 +778,7 @@ func TestRecordingStructuredLogLimit(t *testing.T) {
778778
// maxStructuredBytesPerTrace allows.
779779
// We'll check that some these messages are dropped when collecting the
780780
// recording.
781-
numMessages := 2 * maxStructuredBytesPerTrace / msgSize
781+
numMessages := 2 * defaultMaxStructuredBytesPerTrace / msgSize
782782
for i := 0; i < int(numMessages); i++ {
783783
c := tr.StartSpan(fmt.Sprintf("child %d", i), WithParent(sp))
784784
c.RecordStructured(msg)
@@ -792,7 +792,7 @@ func TestRecordingStructuredLogLimit(t *testing.T) {
792792
trace := sp.i.crdb.GetRecording(tracingpb.RecordingVerbose, false /* finishing */)
793793
// Check that we have two events in the trace (one per span).
794794
require.Less(t, len(trace.appendStructuredEventsRecursively(nil /* buffer */)), int(numMessages))
795-
require.LessOrEqual(t, trace.StructuredRecordsSizeBytes, int64(maxStructuredBytesPerTrace))
795+
require.LessOrEqual(t, trace.StructuredRecordsSizeBytes, int64(defaultMaxStructuredBytesPerTrace))
796796
})
797797
}
798798
}
@@ -1180,8 +1180,8 @@ func TestStructuredRecordingSizeLimit(t *testing.T) {
11801180
require.NoError(t, err)
11811181
record := &tracingpb.StructuredRecord{Payload: p}
11821182
eventSize := record.MemorySize()
1183-
maxEventsPerSpan := maxStructuredBytesPerSpan / eventSize
1184-
maxEventPerTrace := maxStructuredBytesPerTrace / eventSize
1183+
maxEventsPerSpan := defaultMaxStructuredBytesPerSpan / eventSize
1184+
maxEventPerTrace := defaultMaxStructuredBytesPerTrace / eventSize
11851185
// Try to record more events than the per-span limit allows, and check that
11861186
// some get dropped.
11871187
for i := 0; i < maxEventsPerSpan+10; i++ {

pkg/util/tracing/tracer.go

Lines changed: 63 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -50,22 +50,22 @@ const (
5050
maxRecordedSpansPerTrace = 1000
5151
// maxRecordedBytesPerSpan limits the size of unstructured logs in a span.
5252
maxLogBytesPerSpan = 256 * (1 << 10) // 256 KiB
53-
// maxStructuredBytesPerSpan limits the size of structured logs in a span.
53+
// defaultMaxStructuredBytesPerSpan limits the size of structured logs in a span.
5454
// This limit applies to records directly logged into the span; it does not
5555
// apply to records in child span (including structured records copied from
5656
// the child into the parent when the child is dropped because of the number
5757
// of spans limit).
5858
// See also maxStructuredBytesPerTrace.
59-
maxStructuredBytesPerSpan = 10 * (1 << 10) // 10 KiB
60-
// maxStructuredBytesPerTrace limits the total size of structured logs in a
59+
defaultMaxStructuredBytesPerSpan = 10 * (1 << 10) // 10 KiB
60+
// defaultMaxStructuredBytesPerTrace limits the total size of structured logs in a
6161
// trace recording, across all spans. This limit is enforced at the time when
6262
// a span is finished and its recording is copied to the parent, and at the
6363
// time when an open span's recording is collected - which calls into all its
6464
// open children. Thus, if there are multiple open spans that are part of the
6565
// same trace, each one of them can temporarily have up to
66-
// maxStructuredBytesPerTrace worth of messages under it. Each open span is
67-
// also subject to the maxStructuredBytesPerSpan limit.
68-
maxStructuredBytesPerTrace = 1 << 20 // 1 MiB
66+
// defaultMaxStructuredBytesPerTrace worth of messages under it. Each open span is
67+
// also subject to the defaultMaxStructuredBytesPerSpan limit.
68+
defaultMaxStructuredBytesPerTrace = 1 << 20 // 1 MiB
6969

7070
// maxSpanRegistrySize limits the number of local root spans tracked in
7171
// a Tracer's registry.
@@ -189,6 +189,18 @@ var periodicSnapshotInterval = settings.RegisterDurationSetting(
189189
0,
190190
settings.WithPublic)
191191

192+
var structuredBytesLimit = settings.RegisterByteSizeSetting(
193+
settings.ApplicationLevel,
194+
"trace.structured_bytes_per_trace.max",
195+
"maximum size of structured log entries per trace recording",
196+
defaultMaxStructuredBytesPerTrace)
197+
198+
var structuredBytesPerSpanLimit = settings.RegisterByteSizeSetting(
199+
settings.ApplicationLevel,
200+
"trace.structured_bytes_per_span.max",
201+
"maximum size of structured log entries per span",
202+
defaultMaxStructuredBytesPerSpan)
203+
192204
// panicOnUseAfterFinish, if set, causes use of a span after Finish() to panic
193205
// if detected.
194206
var panicOnUseAfterFinish = buildutil.CrdbTestBuild ||
@@ -368,6 +380,16 @@ type Tracer struct {
368380

369381
testing TracerTestingKnobs
370382

383+
// maxStructuredBytesPerTrace limits the total size of structured logs in a
384+
// trace recording. This value is configurable via the
385+
// trace.structured_bytes_per_trace.max cluster setting.
386+
_maxStructuredBytesPerTrace int64 // accessed atomically
387+
388+
// maxStructuredBytesPerSpan limits the size of structured logs in each span.
389+
// This value is configurable via the trace.structured_bytes_per_span.max
390+
// cluster setting.
391+
_maxStructuredBytesPerSpan int64 // accessed atomically
392+
371393
// stack is populated in NewTracer and is printed in assertions related to
372394
// mixing tracers.
373395
stack debugutil.SafeStack
@@ -588,6 +610,30 @@ func (t *Tracer) SetActiveSpansRegistryEnabled(to bool) {
588610
atomic.StoreInt32(&t._activeSpansRegistryEnabled, n)
589611
}
590612

613+
// SetMaxStructuredBytesPerTrace sets the maximum size of structured logs per
614+
// trace recording.
615+
func (t *Tracer) SetMaxStructuredBytesPerTrace(limit int64) {
616+
atomic.StoreInt64(&t._maxStructuredBytesPerTrace, limit)
617+
}
618+
619+
// MaxStructuredBytesPerTrace returns the maximum size of structured logs per
620+
// trace recording.
621+
func (t *Tracer) MaxStructuredBytesPerTrace() int64 {
622+
return atomic.LoadInt64(&t._maxStructuredBytesPerTrace)
623+
}
624+
625+
// SetMaxStructuredBytesPerSpan sets the maximum size of structured logs per
626+
// span.
627+
func (t *Tracer) SetMaxStructuredBytesPerSpan(limit int64) {
628+
atomic.StoreInt64(&t._maxStructuredBytesPerSpan, limit)
629+
}
630+
631+
// MaxStructuredBytesPerSpan returns the maximum size of structured logs per
632+
// span.
633+
func (t *Tracer) MaxStructuredBytesPerSpan() int64 {
634+
return atomic.LoadInt64(&t._maxStructuredBytesPerSpan)
635+
}
636+
591637
// ActiveSpansRegistryEnabled returns true if this tracer is configured
592638
// to register spans with the activeSpansRegistry
593639
func (t *Tracer) ActiveSpansRegistryEnabled() bool {
@@ -606,8 +652,10 @@ func NewTracer() *Tracer {
606652
}
607653

608654
t := &Tracer{
609-
stack: debugutil.Stack(),
610-
activeSpansRegistry: makeSpanRegistry(),
655+
stack: debugutil.Stack(),
656+
activeSpansRegistry: makeSpanRegistry(),
657+
_maxStructuredBytesPerTrace: defaultMaxStructuredBytesPerTrace,
658+
_maxStructuredBytesPerSpan: defaultMaxStructuredBytesPerSpan,
611659
// These might be overridden in NewTracerWithOpt.
612660
panicOnUseAfterFinish: panicOnUseAfterFinish,
613661
debugUseAfterFinish: debugUseAfterFinish,
@@ -785,6 +833,11 @@ func (t *Tracer) configure(ctx context.Context, sv *settings.Values, tracingDefa
785833
otlpCollectorAddr := openTelemetryCollector.Get(sv)
786834
zipkinAddr := ZipkinCollector.Get(sv)
787835
enableRedactable := enableTraceRedactable.Get(sv)
836+
structuredBytesLimitVal := structuredBytesLimit.Get(sv)
837+
structuredBytesPerSpanLimitVal := structuredBytesPerSpanLimit.Get(sv)
838+
839+
t.SetMaxStructuredBytesPerTrace(structuredBytesLimitVal)
840+
t.SetMaxStructuredBytesPerSpan(structuredBytesPerSpanLimitVal)
788841

789842
switch tracingDefault {
790843
case TracingModeFromEnv:
@@ -872,6 +925,8 @@ func (t *Tracer) configure(ctx context.Context, sv *settings.Values, tracingDefa
872925
openTelemetryCollector.SetOnChange(sv, reconfigure)
873926
ZipkinCollector.SetOnChange(sv, reconfigure)
874927
enableTraceRedactable.SetOnChange(sv, reconfigure)
928+
structuredBytesLimit.SetOnChange(sv, reconfigure)
929+
structuredBytesPerSpanLimit.SetOnChange(sv, reconfigure)
875930
}
876931

877932
func createOTLPSpanProcessor(

0 commit comments

Comments
 (0)