diff --git a/pkg/server/node_tenant_test.go b/pkg/server/node_tenant_test.go index 1d02889ea8fe..180ea60a7991 100644 --- a/pkg/server/node_tenant_test.go +++ b/pkg/server/node_tenant_test.go @@ -128,6 +128,7 @@ func TestRedactRecordingForTenant(t *testing.T) { GoroutineID uint64 Finished bool StructuredRecords []tracingpb.StructuredRecord + ChildrenMetadata map[string]tracingpb.OperationMetadata } _ = (*calcifiedRecordedSpan)((*tracingpb.RecordedSpan)(nil)) }) diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index 314e5d62f648..a4d810a4fe50 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -168,6 +168,14 @@ type recordingState struct { // // The spans are not maintained in a particular order. finishedChildren []tracingpb.RecordedSpan + + // childrenMetadata is a mapping from operation to the aggregated metadata of + // that operation. + // + // When a child of this span is Finish()ed, it updates the map with all the + // children in its Recording. childrenMetadata therefore provides a bucketed + // view of the various operations that are being traced as part of a span. + childrenMetadata map[string]tracingpb.OperationMetadata } // makeSizeLimitedBuffer creates a sizeLimitedBuffer. @@ -377,8 +385,20 @@ func (s *crdbSpan) getRecordingImpl( } } +// rollupChildrenMetadata combines the OperationMetadata in `from` into `to`. +func rollupChildrenMetadata( + to map[string]tracingpb.OperationMetadata, from map[string]tracingpb.OperationMetadata, +) { + for op, metadata := range from { + to[op] = to[op].Combine(metadata) + } +} + // getVerboseRecording returns the Span's recording, including its children. // +// Each RecordedSpan in the Recording contains the ChildrenMetadata of all the +// children, both finished and open, in the spans' subtree. +// // finishing indicates whether s is in the process of finishing. If it isn't, // the recording will include an "_unfinished" tag. func (s *crdbSpan) getVerboseRecording( @@ -388,21 +408,53 @@ func (s *crdbSpan) getVerboseRecording( return nil // noop span } - s.mu.Lock() - // The capacity here is approximate since we don't know how many - // grandchildren there are. - result := make(tracingpb.Recording, 0, 1+len(s.mu.openChildren)+len(s.mu.recording.finishedChildren)) - result = append(result, s.getRecordingNoChildrenLocked(tracingpb.RecordingVerbose, finishing)) - result = append(result, s.mu.recording.finishedChildren...) - - for _, child := range s.mu.openChildren { - if child.collectRecording || includeDetachedChildren { - sp := child.Span.i.crdb - result = append(result, sp.getVerboseRecording(includeDetachedChildren, false /* finishing */)...) + var result tracingpb.Recording + var childrenMetadata map[string]tracingpb.OperationMetadata + { + s.mu.Lock() + // The capacity here is approximate since we don't know how many + // grandchildren there are. + result = make(tracingpb.Recording, 0, 1+len(s.mu.openChildren)+len(s.mu.recording.finishedChildren)) + result = append(result, s.getRecordingNoChildrenLocked(tracingpb.RecordingVerbose, finishing)) + result = append(result, s.mu.recording.finishedChildren...) + + childrenMetadata = make(map[string]tracingpb.OperationMetadata) + + // Copy over the OperationMetadata collected from s' finished children. + rollupChildrenMetadata(childrenMetadata, s.mu.recording.childrenMetadata) + + // We recurse on s' open children to get their verbose recordings, and to + // aggregate OperationMetadata from their children, both finished and open. + for _, openChild := range s.mu.openChildren { + if openChild.collectRecording || includeDetachedChildren { + openChildSp := openChild.Span.i.crdb + openChildRecording := openChildSp.getVerboseRecording(includeDetachedChildren, false /* finishing */) + result = append(result, openChildRecording...) + + // Record an entry for openChilds' OperationMetadata. + rootOpenChild := openChildRecording[0] + prevMetadata := childrenMetadata[rootOpenChild.Operation] + prevMetadata.Count++ + prevMetadata.ContainsUnfinished = !rootOpenChild.Finished + prevMetadata.Duration += rootOpenChild.Duration + childrenMetadata[rootOpenChild.Operation] = prevMetadata + + // Copy over the OperationMetadata collected recursively from openChilds' + // children. + rollupChildrenMetadata(childrenMetadata, openChildRecording[0].ChildrenMetadata) + } } } + s.mu.Unlock() + // Copy over the OperationMetadata collected from s' children into the root of + // the recording. + if len(childrenMetadata) != 0 { + rootSpan := &result[0] + rootSpan.ChildrenMetadata = childrenMetadata + } + // Sort the spans by StartTime, except the first Span (the root of this // recording) which stays in place. toSort := sortPool.Get().(*tracingpb.Recording) // avoids allocations in sort.Sort @@ -413,59 +465,35 @@ func (s *crdbSpan) getVerboseRecording( return result } -// getStructuredRecording returns the structured events in this span and -// in all the children. The results are returned as a Recording for the caller's -// convenience (and for optimizing memory allocations). The Recording will be -// nil if there are no structured events. If not nil, the Recording will have -// exactly one span corresponding to the receiver, will all events handing from +// getStructuredRecording returns the structured events in this span and in all +// the children. The results are returned as a Recording for the caller's +// convenience (and for optimizing memory allocations). The Recording will have +// exactly one span corresponding to the receiver, with all events hanging from // this span (even if the events had been recorded on different spans). +// This span will also have a `childrenMetadata` map that will contain an entry +// for all children in s' Recording. // // The caller does not take ownership of the events. func (s *crdbSpan) getStructuredRecording(includeDetachedChildren bool) tracingpb.Recording { s.mu.Lock() defer s.mu.Unlock() - buffer := make([]*tracingpb.StructuredRecord, 0, 3) - for _, c := range s.mu.recording.finishedChildren { - for i := range c.StructuredRecords { - buffer = append(buffer, &c.StructuredRecords[i]) - } - } - for _, c := range s.mu.openChildren { - if c.collectRecording || includeDetachedChildren { - sp := c.Span.i.crdb - buffer = sp.getStructuredEventsRecursively(buffer, includeDetachedChildren) - } - } - - if len(buffer) == 0 && s.mu.recording.structured.Len() == 0 { - // Optimize out the allocations below. - return nil - } res := s.getRecordingNoChildrenLocked( tracingpb.RecordingStructured, false, // finishing - since we're only asking for the structured recording, the argument doesn't matter ) - // If necessary, grow res.StructuredRecords to have space for buffer. - var reservedSpace []tracingpb.StructuredRecord - if cap(res.StructuredRecords)-len(res.StructuredRecords) < len(buffer) { - // res.StructuredRecords does not have enough capacity to accommodate the - // elements of buffer. We allocate a new, larger array and copy over the old - // entries. - old := res.StructuredRecords - res.StructuredRecords = make([]tracingpb.StructuredRecord, len(old)+len(buffer)) - copy(res.StructuredRecords, old) - reservedSpace = res.StructuredRecords[len(old):] - } else { - // res.StructuredRecords has enough capacity for buffer. We extend it in - // place. - oldLen := len(res.StructuredRecords) - res.StructuredRecords = res.StructuredRecords[:oldLen+len(buffer)] - reservedSpace = res.StructuredRecords[oldLen:] - } - for i, e := range buffer { - reservedSpace[i] = *e - } + + // Recursively fetch the StructuredEvents for s' and its children, both + // finished and open. + res.StructuredRecords = s.getStructuredEventsRecursivelyLocked(res.StructuredRecords[:0], + includeDetachedChildren) + + // Recursively fetch the OperationMetadata for s' children, both finished and + // open. + res.ChildrenMetadata = make(map[string]tracingpb.OperationMetadata) + s.getChildrenMetadataRecursivelyLocked(res.ChildrenMetadata, + false /* includeRootMetadata */, includeDetachedChildren) + return tracingpb.Recording{res} } @@ -497,6 +525,8 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRecording tracingpb.Recordi return } + rootChild := &childRecording[0] + // Depending on the type of recording, we either keep all the information // received, or only the structured events. switch s.recordingType() { @@ -505,7 +535,7 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRecording tracingpb.Recordi // usually already the case, except with DistSQL traces where remote // processors run in spans that FollowFrom an RPC Span that we don't // collect. - childRecording[0].ParentSpanID = s.spanID + rootChild.ParentSpanID = s.spanID if len(s.mu.recording.finishedChildren)+len(childRecording) <= maxRecordedSpansPerTrace { s.mu.recording.finishedChildren = append(s.mu.recording.finishedChildren, childRecording...) @@ -516,17 +546,44 @@ func (s *crdbSpan) recordFinishedChildrenLocked(childRecording tracingpb.Recordi // records by falling through. fallthrough case tracingpb.RecordingStructured: - for ci := range childRecording { - child := &childRecording[ci] - for i := range child.StructuredRecords { - s.recordInternalLocked(&child.StructuredRecords[i], &s.mu.recording.structured) - } + if len(childRecording) != 1 { + panic(fmt.Sprintf("RecordingStructured has %d recordings; expected 1", len(childRecording))) + } + + for i := range rootChild.StructuredRecords { + s.recordInternalLocked(&rootChild.StructuredRecords[i], &s.mu.recording.structured) } case tracingpb.RecordingOff: break default: panic(fmt.Sprintf("unrecognized recording mode: %v", s.recordingType())) } + + // Update s' ChildrenMetadata to capture all the spans in `childRecording`. + // + // As an example where we are done finishing `child`: + // + // parent + // child + // grandchild + // + // `parent` will have: + // {child: 2s, grandchild: 1s} + // + // Record finished rootChilds' metadata. + s.mu.recording.childrenMetadata[rootChild.Operation] = + s.mu.recording.childrenMetadata[rootChild.Operation].Combine( + tracingpb.OperationMetadata{ + Count: 1, + Duration: rootChild.Duration, + ContainsUnfinished: false, + }) + + // Record the metadata of rootChilds' children, both finished and open. + // + // GetRecording(...) is responsible for recursively capturing the metadata for + // rootChilds' open and finished children. + rollupChildrenMetadata(s.mu.recording.childrenMetadata, rootChild.ChildrenMetadata) } func (s *crdbSpan) setTagLocked(key string, value attribute.Value) { @@ -698,35 +755,73 @@ func (s *crdbSpan) recordInternalLocked(payload memorySizable, buffer *sizeLimit buffer.AddLast(payload) } -// getStructuredEventsRecursively returns the structured events accumulated by -// this span and its finished and still-open children. -func (s *crdbSpan) getStructuredEventsRecursively( - buffer []*tracingpb.StructuredRecord, includeDetachedChildren bool, -) []*tracingpb.StructuredRecord { - s.mu.Lock() - defer s.mu.Unlock() +// getStructuredEventsRecursivelyLocked returns the structured events +// accumulated by s' and its finished and still-open children. +func (s *crdbSpan) getStructuredEventsRecursivelyLocked( + buffer []tracingpb.StructuredRecord, includeDetachedChildren bool, +) []tracingpb.StructuredRecord { buffer = s.getStructuredEventsLocked(buffer) for _, c := range s.mu.openChildren { if c.collectRecording || includeDetachedChildren { sp := c.Span.i.crdb - buffer = sp.getStructuredEventsRecursively(buffer, includeDetachedChildren) + sp.mu.Lock() + buffer = sp.getStructuredEventsRecursivelyLocked(buffer, includeDetachedChildren) + sp.mu.Unlock() } } for _, c := range s.mu.recording.finishedChildren { for i := range c.StructuredRecords { - buffer = append(buffer, &c.StructuredRecords[i]) + buffer = append(buffer, c.StructuredRecords[i]) } } return buffer } +// getChildrenMetadataRecursivelyLocked populates `childrenMetadata` with +// OperationMetadata entries for all of s' children (open and finished), +// recursively. +// +// The method also populates `childrenMetadata` with an entry for the receiver +// if `includeRootMetadata` is true. +func (s *crdbSpan) getChildrenMetadataRecursivelyLocked( + childrenMetadata map[string]tracingpb.OperationMetadata, + includeRootMetadata, includeDetachedChildren bool, +) { + if includeRootMetadata { + // Record an entry for s' metadata. + prevMetadata := childrenMetadata[s.operation] + prevMetadata.Count++ + if s.mu.duration == -1 { + prevMetadata.Duration += timeutil.Since(s.startTime) + prevMetadata.ContainsUnfinished = true + } else { + prevMetadata.Duration += s.mu.duration + } + childrenMetadata[s.operation] = prevMetadata + } + + // Copy over s' Finish()ed children metadata. + rollupChildrenMetadata(childrenMetadata, s.mu.recording.childrenMetadata) + + // For each of s' open children, recurse to collect their metadata. + for _, c := range s.mu.openChildren { + if c.collectRecording || includeDetachedChildren { + sp := c.Span.i.crdb + sp.mu.Lock() + sp.getChildrenMetadataRecursivelyLocked(childrenMetadata, + true /*includeRootMetadata */, includeDetachedChildren) + sp.mu.Unlock() + } + } +} + func (s *crdbSpan) getStructuredEventsLocked( - buffer []*tracingpb.StructuredRecord, -) []*tracingpb.StructuredRecord { + buffer []tracingpb.StructuredRecord, +) []tracingpb.StructuredRecord { numEvents := s.mu.recording.structured.Len() for i := 0; i < numEvents; i++ { event := s.mu.recording.structured.Get(i).(*tracingpb.StructuredRecord) - buffer = append(buffer, event) + buffer = append(buffer, *event) } return buffer } diff --git a/pkg/util/tracing/span.go b/pkg/util/tracing/span.go index 897409d4340e..56ae26ff90f9 100644 --- a/pkg/util/tracing/span.go +++ b/pkg/util/tracing/span.go @@ -672,8 +672,9 @@ func (sp *Span) reset( openChildren: h.childrenAlloc[:0], goroutineID: goroutineID, recording: recordingState{ - logs: makeSizeLimitedBuffer(maxLogBytesPerSpan, nil /* scratch */), - structured: makeSizeLimitedBuffer(maxStructuredBytesPerSpan, h.structuredEventsAlloc[:]), + logs: makeSizeLimitedBuffer(maxLogBytesPerSpan, nil /* scratch */), + structured: makeSizeLimitedBuffer(maxStructuredBytesPerSpan, h.structuredEventsAlloc[:]), + childrenMetadata: make(map[string]tracingpb.OperationMetadata), }, tags: h.tagsAlloc[:0], } diff --git a/pkg/util/tracing/span_test.go b/pkg/util/tracing/span_test.go index 8718d2868e0f..6f9755e3bde5 100644 --- a/pkg/util/tracing/span_test.go +++ b/pkg/util/tracing/span_test.go @@ -372,6 +372,7 @@ func TestChildSpanRegisteredWithRecordingParent(t *testing.T) { rec := sp.GetRecording(tracingpb.RecordingStructured) require.Len(t, rec, 1) require.Len(t, rec[0].StructuredRecords, 1) + require.Len(t, rec[0].ChildrenMetadata, 1) } // TestRecordingMaxSpans verifies that recordings don't grow over the limit. @@ -691,6 +692,148 @@ func TestStructureRecording(t *testing.T) { } } +// TestVerboseRecordingFinishedChildrenDurations tests that the VerboseRecording +// of a span includes the `operation : duration` mapping of all children +// (finished and open) in that recording. +func TestVerboseRecordingFinishedChildrenDurations(t *testing.T) { + checkChildrenMetadata := func( + actualOperations map[string]tracingpb.OperationMetadata, + expectedOperations map[string]tracingpb.OperationMetadata) { + t.Helper() + require.Len(t, actualOperations, len(expectedOperations)) + for opName, actualMetadata := range actualOperations { + expectedMetadata, ok := expectedOperations[opName] + require.True(t, ok) + require.Greater(t, actualMetadata.Duration, time.Duration(0)) + require.Equal(t, expectedMetadata.Count, actualMetadata.Count) + require.Equal(t, expectedMetadata.ContainsUnfinished, actualMetadata.ContainsUnfinished) + } + } + tr := NewTracer() + root := tr.StartSpan("root", WithRecording(tracingpb.RecordingVerbose)) + child := tr.StartSpan("child", WithParent(root)) + gc := tr.StartSpan("grandchild", WithParent(child)) + child2 := tr.StartSpan("child2", WithParent(root)) + + // grandchild has no children, and so the recording should contain no + // metadata. + rec := gc.FinishAndGetConfiguredRecording() + require.Len(t, rec, 1) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{}) + + rec = child.GetConfiguredRecording() + require.Len(t, rec, 2) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "grandchild": {Count: 1}, + }) + + // Now, let's Finish() child2, but leave child as an open child of root. + child2.Finish() + + // Root should have entries for Finish()ed `child2`, open child `child`, and + // its Finish()ed `grandchild`. + rec = root.GetConfiguredRecording() + require.Len(t, rec, 4) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "child2": {Count: 1}, + "child": {Count: 1, ContainsUnfinished: true}, + "grandchild": {Count: 1}, + }) + + // Create another span with the same opName. + childWithSameOpName := tr.StartSpan("child", WithParent(root)) + childWithSameOpName.Finish() + rec = root.GetConfiguredRecording() + require.Len(t, rec, 5) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "child2": {Count: 1}, + "child": {Count: 2, ContainsUnfinished: true}, + "grandchild": {Count: 1}, + }) + + // Finish child, and re-check root's recording. + child.Finish() + rec = root.FinishAndGetConfiguredRecording() + require.Len(t, rec, 5) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "child2": {Count: 1}, + "child": {Count: 2}, + "grandchild": {Count: 1}, + }) +} + +// TestStructuredRecordingFinishedChildrenDurations tests that the +// StructuredRecording of a span includes the `operation : duration` mapping of +// all children (finished and open) in that recording. +func TestStructuredRecordingFinishedChildrenDurations(t *testing.T) { + checkChildrenMetadata := func( + actualOperations map[string]tracingpb.OperationMetadata, + expectedOperations map[string]tracingpb.OperationMetadata) { + t.Helper() + require.Len(t, actualOperations, len(expectedOperations)) + for opName, actualMetadata := range actualOperations { + expectedMetadata, ok := expectedOperations[opName] + require.True(t, ok) + require.Greater(t, actualMetadata.Duration, time.Duration(0)) + require.Equal(t, expectedMetadata.Count, actualMetadata.Count) + require.Equal(t, expectedMetadata.ContainsUnfinished, actualMetadata.ContainsUnfinished) + } + } + tr := NewTracer() + root := tr.StartSpan("root", WithRecording(tracingpb.RecordingStructured)) + child := tr.StartSpan("child", WithParent(root)) + gc := tr.StartSpan("grandchild", WithParent(child)) + child2 := tr.StartSpan("child2", WithParent(root)) + + // grandchild has no children, and so the recording should contain no + // metadata. + rec := gc.FinishAndGetConfiguredRecording() + require.Len(t, rec, 1) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{}) + + // child has no StructuredEvents, but it should return information about its + // children. + rec = child.GetConfiguredRecording() + require.Len(t, rec, 1) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "grandchild": {Count: 1}, + }) + + // Now, let's Finish() child2, but leave child as an open child of root. + child2.Finish() + + // Root should have entries for Finish()ed `child2`, open child `child`, and + // its Finish()ed `grandchild`. + rec = root.GetConfiguredRecording() + require.Len(t, rec, 1) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "child2": {Count: 1}, + "child": {Count: 1, ContainsUnfinished: true}, + "grandchild": {Count: 1}, + }) + + // Create another span with the same opName. + childWithSameOpName := tr.StartSpan("child", WithParent(root)) + childWithSameOpName.Finish() + rec = root.GetConfiguredRecording() + require.Len(t, rec, 1) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "child2": {Count: 1}, + "child": {Count: 2, ContainsUnfinished: true}, + "grandchild": {Count: 1}, + }) + + // Finish child, and re-check root's recording. + child.Finish() + rec = root.FinishAndGetConfiguredRecording() + require.Len(t, rec, 1) + checkChildrenMetadata(rec[0].ChildrenMetadata, map[string]tracingpb.OperationMetadata{ + "child2": {Count: 1}, + "child": {Count: 2}, + "grandchild": {Count: 1}, + }) +} + // Test that a child span that's still open at the time when // parent.FinishAndGetRecording() is called is included in the parent's // recording. @@ -823,3 +966,88 @@ func TestEventListenerNotifiedWithoutHoldingSpanMutex(t *testing.T) { sp.RecordStructured(&types.Int32Value{Value: 5}) require.Equal(t, 1, rootEventListener.eventsSeen) } + +// TestFinishedChildrenMetadata tests that on Finish() the parent span's +// `childrenMetadata` map captures all the children in the recording rooted at +// the finished span. +func TestFinishedChildrenMetadata(t *testing.T) { + checkChildrenMetadata := func( + sp *Span, expectedOperations map[string]tracingpb.OperationMetadata) { + t.Helper() + c := sp.i.crdb + c.mu.Lock() + defer c.mu.Unlock() + require.Len(t, c.mu.recording.childrenMetadata, len(expectedOperations)) + for opName, actualMetadata := range c.mu.recording.childrenMetadata { + expectedMetadata, ok := expectedOperations[opName] + require.True(t, ok) + require.Greater(t, actualMetadata.Duration, time.Duration(0)) + require.Equal(t, expectedMetadata.Count, actualMetadata.Count) + require.Equal(t, expectedMetadata.ContainsUnfinished, actualMetadata.ContainsUnfinished) + } + } + + fn := func(recordingMode tracingpb.RecordingType) { + tr := NewTracer() + root := tr.StartSpan("root", WithRecording(recordingMode)) + child := tr.StartSpan("child", WithParent(root)) + + gc1 := tr.StartSpan("grandchild1", WithParent(child)) + ggc1 := tr.StartSpan("greatgrandchild1", WithParent(gc1)) + + gc2 := tr.StartSpan("grandchild2", WithParent(child)) + ggc2 := tr.StartSpan("greatgrandchild2", WithParent(gc2)) + + // First let's Finish() the lowest children. This should mean that their + // parents have entries for their duration. + ggc1.Finish() + checkChildrenMetadata(gc1, map[string]tracingpb.OperationMetadata{ + "greatgrandchild1": {Count: 1}, + }) + + ggc2.Finish() + checkChildrenMetadata(gc2, map[string]tracingpb.OperationMetadata{ + "greatgrandchild2": {Count: 1}, + }) + + // Finish() one of the grand children. + gc1.Finish() + checkChildrenMetadata(child, map[string]tracingpb.OperationMetadata{ + "grandchild1": {Count: 1}, + "greatgrandchild1": {Count: 1}, + }) + + // Now Finish() `child` since it has both finished and open children at this + // point. We expect to see metadata of all children (finished + open) rooted + // in childs' recording to be copied into root. + child.Finish() + checkChildrenMetadata(root, map[string]tracingpb.OperationMetadata{ + "child": {Count: 1}, + "grandchild1": {Count: 1}, + "greatgrandchild1": {Count: 1}, + "grandchild2": {Count: 1, ContainsUnfinished: true}, + "greatgrandchild2": {Count: 1}, + }) + + // gc2 should now be a root span in the registry since it was open when + // `child` was finished. Finishing it should have no effect on roots' + // recorded metadata. + gc2.Finish() + checkChildrenMetadata(root, map[string]tracingpb.OperationMetadata{ + "child": {Count: 1}, + "grandchild1": {Count: 1}, + "greatgrandchild1": {Count: 1}, + "grandchild2": {Count: 1, ContainsUnfinished: true}, + "greatgrandchild2": {Count: 1}, + }) + root.Finish() + } + + t.Run("verbose-recording", func(t *testing.T) { + fn(tracingpb.RecordingVerbose) + }) + + t.Run("structured-recording", func(t *testing.T) { + fn(tracingpb.RecordingStructured) + }) +} diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index 0e031163d748..9dec1b83e50b 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -86,7 +86,12 @@ func TestTracerRecording(t *testing.T) { } // Initial recording of this fresh (real) span. - require.Nil(t, s1.GetRecording(tracingpb.RecordingStructured)) + rec := s1.GetRecording(tracingpb.RecordingStructured) + require.Len(t, rec, 1) + require.Nil(t, rec[0].Logs) + require.Nil(t, rec[0].Tags) + require.Empty(t, rec[0].ChildrenMetadata) + require.Empty(t, rec[0].StructuredRecords) s1.RecordStructured(&types.Int32Value{Value: 5}) if err := CheckRecording(s1.GetRecording(tracingpb.RecordingStructured), ` diff --git a/pkg/util/tracing/tracingpb/recorded_span.go b/pkg/util/tracing/tracingpb/recorded_span.go index dc2a1667d168..1c43d0ac86a4 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.go +++ b/pkg/util/tracing/tracingpb/recorded_span.go @@ -120,3 +120,11 @@ func (tg *TagGroup) FindTag(key string) (string, bool) { } return "", false } + +// Combine returns the sum of m and other. +func (m OperationMetadata) Combine(other OperationMetadata) OperationMetadata { + m.Count += other.Count + m.ContainsUnfinished = m.ContainsUnfinished || other.ContainsUnfinished + m.Duration += other.Duration + return m +} diff --git a/pkg/util/tracing/tracingpb/recorded_span.proto b/pkg/util/tracing/tracingpb/recorded_span.proto index 0b79e43621aa..86bd64d34f13 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.proto +++ b/pkg/util/tracing/tracingpb/recorded_span.proto @@ -43,6 +43,19 @@ message StructuredRecord { google.protobuf.Any payload = 2; } +// OperationMetadata captures information corresponding to the operation that +// a span is started with. +message OperationMetadata { + // Duration represents the total time spent by spans tracing the operation. + int64 duration = 1 [(gogoproto.casttype) = "time.Duration"]; + // Count represents the number of spans tracing the operation. + int64 count = 2; + // ContainsUnfinished is true if the OperationMetadata contains information + // about unfinished spans tracing the operation. + bool contains_unfinished = 3; +} + + // RecordedSpan is the data recorded by a trace span. It // needs to be able to cross RPC boundaries so that the // complete recording of the trace can be constructed. @@ -119,7 +132,14 @@ message RecordedSpan { // DeprecatedInternalStructured only stores the Payloads. repeated StructuredRecord structured_records = 14 [(gogoproto.nullable) = false]; - // Next ID: 19 + + // ChildrenMetadata is a mapping from operation to the aggregated metadata of + // that operation. + // + // When a child of this span is Finish()ed, it updates the map with all the + // children in its Recording. ChildrenMetadata therefore provides a bucketed + // view of the various operations that are being traced as part of a span. + map children_metadata = 19 [(gogoproto.nullable) = false]; reserved 5,10,11; }