diff --git a/pkg/kv/kvserver/replica_application_decoder.go b/pkg/kv/kvserver/replica_application_decoder.go index 399efe911a36..b15362d6d2f5 100644 --- a/pkg/kv/kvserver/replica_application_decoder.go +++ b/pkg/kv/kvserver/replica_application_decoder.go @@ -152,9 +152,9 @@ func (d *replicaDecoder) createTracingSpans(ctx context.Context) { cmd.ctx, cmd.sp = d.r.AmbientContext.Tracer.StartSpanCtx( ctx, opName, - // NB: we are lying here - we are not actually going to propagate - // the recording towards the root. That seems ok. - tracing.WithParentAndManualCollection(spanMeta), + // NB: Nobody is collecting the recording of this span; we have no + // mechanism for it. + tracing.WithRemoteParent(spanMeta), tracing.WithFollowsFrom(), ) } diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index c2bf1f82af8c..05370033ffaa 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -329,7 +329,7 @@ func (p *pendingLeaseRequest) requestLeaseAsync( ctx, sp = tr.StartSpanCtx( ctx, opName, - tracing.WithParentAndAutoCollection(parentSp), + tracing.WithParent(parentSp), tracing.WithFollowsFrom(), tagsOpt, ) diff --git a/pkg/migration/migrationmanager/manager_external_test.go b/pkg/migration/migrationmanager/manager_external_test.go index d661db3c4c20..a926bc6d84f0 100644 --- a/pkg/migration/migrationmanager/manager_external_test.go +++ b/pkg/migration/migrationmanager/manager_external_test.go @@ -161,15 +161,13 @@ RETURNING id;`).Scan(&secondID)) }() testutils.SucceedsSoon(t, func() error { - // TODO(yuzefovich): this check is quite unfortunate since it relies on - // the assumption that all recordings from the child spans are imported - // into the tracer. However, this is not the case for the DistSQL - // processors where child spans are created with - // WithParentAndManualCollection option which requires explicitly - // importing the recordings from the children. This only happens when - // the execution flow is drained which cannot happen until we close - // the 'unblock' channel, and this we cannot do until we see the - // expected message in the trace. + // TODO(yuzefovich): this check is quite unfortunate since it relies on the + // assumption that all recordings from the child spans are imported into the + // tracer. However, this is not the case for the DistSQL processors whose + // recordings require explicit importing. This only happens when the + // execution flow is drained which cannot happen until we close the + // 'unblock' channel, and this we cannot do until we see the expected + // message in the trace. // // At the moment it works in a very fragile manner (by making sure that // no processors actually create their own spans). Instead, a different diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 9ea988f19b49..15af0eb45c08 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1353,7 +1353,7 @@ CREATE TABLE crdb_internal.node_inflight_trace_spans ( "only users with the admin role are allowed to read crdb_internal.node_inflight_trace_spans") } return p.ExecCfg().AmbientCtx.Tracer.VisitSpans(func(span tracing.RegistrySpan) error { - for _, rec := range span.GetRecording(tracing.RecordingVerbose) { + for _, rec := range span.GetFullRecording(tracing.RecordingVerbose) { traceID := rec.TraceID parentSpanID := rec.ParentSpanID spanID := rec.SpanID diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 85af271b70ac..8b7f6eed6816 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -700,13 +700,13 @@ func TestDistSQLFlowsVirtualTables(t *testing.T) { // // Traces on node1: // ------------- -// root <-- traceID1 -// root.child <-- traceID1 -// root.child.remotechild <-- traceID1 +// root <-- traceID1 +// root.child <-- traceID1 +// root.child.detached_child <-- traceID1 // // Traces on node2: // ------------- -// root.child.remotechild2 <-- traceID1 +// root.child.remotechild <-- traceID1 // root.child.remotechilddone <-- traceID1 // root2 <-- traceID2 // root2.child <-- traceID2 @@ -717,22 +717,22 @@ func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, func()) { time.Sleep(10 * time.Millisecond) // Start a child span on "node 1". - child := t1.StartSpan("root.child", tracing.WithParentAndAutoCollection(root)) + child := t1.StartSpan("root.child", tracing.WithParent(root)) // Sleep a bit so that everything that comes afterwards has higher timestamps // than the one we just assigned. Otherwise the sorting is not deterministic. time.Sleep(10 * time.Millisecond) // Start a forked child span on "node 1". - childRemoteChild := t1.StartSpan("root.child.remotechild", tracing.WithParentAndManualCollection(child.Meta())) + childDetachedChild := t1.StartSpan("root.child.detached_child", tracing.WithParent(child), tracing.WithDetachedRecording()) // Start a remote child span on "node 2". - childRemoteChild2 := t2.StartSpan("root.child.remotechild2", tracing.WithParentAndManualCollection(child.Meta())) + childRemoteChild := t2.StartSpan("root.child.remotechild", tracing.WithRemoteParent(child.Meta())) time.Sleep(10 * time.Millisecond) // Start another remote child span on "node 2" that we finish. - childRemoteChildFinished := t2.StartSpan("root.child.remotechilddone", tracing.WithParentAndManualCollection(child.Meta())) + childRemoteChildFinished := t2.StartSpan("root.child.remotechilddone", tracing.WithRemoteParent(child.Meta())) child.ImportRemoteSpans(childRemoteChildFinished.FinishAndGetRecording(tracing.RecordingVerbose)) // Start another remote child span on "node 2" that we finish. This will have @@ -740,10 +740,10 @@ func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, func()) { root2 := t2.StartSpan("root2", tracing.WithRecording(tracing.RecordingVerbose)) // Start a child span on "node 2". - child2 := t2.StartSpan("root2.child", tracing.WithParentAndAutoCollection(root2)) + child2 := t2.StartSpan("root2.child", tracing.WithParent(root2)) return root.TraceID(), func() { - for _, span := range []*tracing.Span{root, child, childRemoteChild, - childRemoteChild2, root2, child2} { + for _, span := range []*tracing.Span{root, child, childDetachedChild, + childRemoteChild, root2, child2} { span.Finish() } } @@ -766,13 +766,16 @@ func TestClusterInflightTracesVirtualTable(t *testing.T) { traceID, cleanup := setupTraces(node1Tracer, node2Tracer) defer cleanup() + // The cluster_inflight_traces table is magic and only returns results when + // the query contains an index constraint. + t.Run("no-index-constraint", func(t *testing.T) { sqlDB.CheckQueryResults(t, `SELECT * from crdb_internal.cluster_inflight_traces`, [][]string{}) }) t.Run("with-index-constraint", func(t *testing.T) { // We expect there to be 3 tracing.Recordings rooted at - // root, root.child.remotechild, root.child.remotechild2. + // root and root.child.remotechild. expectedRows := []struct { traceID int nodeID int @@ -781,10 +784,6 @@ func TestClusterInflightTracesVirtualTable(t *testing.T) { traceID: int(traceID), nodeID: 1, }, - { - traceID: int(traceID), - nodeID: 1, - }, { traceID: int(traceID), nodeID: 2, @@ -799,8 +798,8 @@ func TestClusterInflightTracesVirtualTable(t *testing.T) { require.NoError(t, rows.Scan(&traceID, &nodeID, &traceStr, &jaegarJSON)) require.Less(t, rowIdx, len(expectedRows)) expected := expectedRows[rowIdx] - require.Equal(t, expected.nodeID, nodeID) require.Equal(t, expected.traceID, traceID) + require.Equal(t, expected.nodeID, nodeID) require.NotEmpty(t, traceStr) require.NotEmpty(t, jaegarJSON) rowIdx++ diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index eaeec0a6b547..f344fa388af2 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -253,13 +253,13 @@ func (ds *ServerImpl) setupFlow( // TODO(andrei): localState.IsLocal is not quite the right thing to use. // If that field is unset, we might still want to create a child span if // this flow is run synchronously. - ctx, sp = ds.Tracer.StartSpanCtx(ctx, opName, tracing.WithParentAndAutoCollection(parentSpan)) + ctx, sp = ds.Tracer.StartSpanCtx(ctx, opName, tracing.WithParent(parentSpan)) } else { // We use FollowsFrom because the flow's span outlives the SetupFlow request. ctx, sp = ds.Tracer.StartSpanCtx( ctx, opName, - tracing.WithParentAndAutoCollection(parentSpan), + tracing.WithParent(parentSpan), tracing.WithFollowsFrom(), ) } diff --git a/pkg/sql/execinfra/processorsbase.go b/pkg/sql/execinfra/processorsbase.go index 18f3266de372..8e14670a40c3 100644 --- a/pkg/sql/execinfra/processorsbase.go +++ b/pkg/sql/execinfra/processorsbase.go @@ -834,7 +834,12 @@ func (pb *ProcessorBase) AppendTrailingMeta(meta execinfrapb.ProducerMetadata) { // ProcessorSpan creates a child span for a processor (if we are doing any // tracing). The returned span needs to be finished using tracing.FinishSpan. func ProcessorSpan(ctx context.Context, name string) (context.Context, *tracing.Span) { - return tracing.ChildSpanRemote(ctx, name) + sp := tracing.SpanFromContext(ctx) + if sp == nil { + return ctx, nil + } + return sp.Tracer().StartSpanCtx(ctx, name, + tracing.WithParent(sp), tracing.WithDetachedRecording()) } // StartInternal prepares the ProcessorBase for execution. It returns the diff --git a/pkg/sql/sem/builtins/generator_builtins.go b/pkg/sql/sem/builtins/generator_builtins.go index fe3142805624..e3846488e8d3 100644 --- a/pkg/sql/sem/builtins/generator_builtins.go +++ b/pkg/sql/sem/builtins/generator_builtins.go @@ -1816,7 +1816,7 @@ func (p *payloadsForSpanGenerator) Start(_ context.Context, _ *kv.Txn) error { // managing the iterator's position needs to start at -1 instead of 0. p.payloadIndex = -1 - rec := p.span.GetRecording(tracing.RecordingStructured) + rec := p.span.GetFullRecording(tracing.RecordingStructured) if rec == nil { // No structured records. return nil diff --git a/pkg/sql/tests/tracing_sql_test.go b/pkg/sql/tests/tracing_sql_test.go index a00855b1e005..6995f5c2bacb 100644 --- a/pkg/sql/tests/tracing_sql_test.go +++ b/pkg/sql/tests/tracing_sql_test.go @@ -46,11 +46,11 @@ func TestSetTraceSpansVerbosityBuiltin(t *testing.T) { defer root.Finish() require.False(t, root.IsVerbose()) - child := tr.StartSpan("root.child", tracing.WithParentAndAutoCollection(root)) + child := tr.StartSpan("root.child", tracing.WithParent(root)) defer child.Finish() require.False(t, child.IsVerbose()) - childChild := tr.StartSpan("root.child.child", tracing.WithParentAndAutoCollection(child)) + childChild := tr.StartSpan("root.child.child", tracing.WithParent(child)) defer childChild.Finish() require.False(t, childChild.IsVerbose()) @@ -79,7 +79,7 @@ func TestSetTraceSpansVerbosityBuiltin(t *testing.T) { require.False(t, childChild.IsVerbose()) // New child of verbose child span should also be verbose by default. - newChild := tr.StartSpan("root.child.newchild", tracing.WithParentAndAutoCollection(root)) + newChild := tr.StartSpan("root.child.newchild", tracing.WithParent(root)) defer newChild.Finish() require.True(t, newChild.IsVerbose()) diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index a65db08b61d6..5e48ee123a21 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -149,7 +149,7 @@ func (tc *TestCluster) stopServers(ctx context.Context) { var buf strings.Builder fmt.Fprintf(&buf, "unexpectedly found %d active spans:\n", len(sps)) for _, sp := range sps { - fmt.Fprintln(&buf, sp.GetRecording(tracing.RecordingVerbose)) + fmt.Fprintln(&buf, sp.GetFullRecording(tracing.RecordingVerbose)) fmt.Fprintln(&buf) } return errors.Newf("%s", buf.String()) diff --git a/pkg/util/stop/stopper_test.go b/pkg/util/stop/stopper_test.go index 368bee539ae0..950af2d130cc 100644 --- a/pkg/util/stop/stopper_test.go +++ b/pkg/util/stop/stopper_test.go @@ -733,7 +733,7 @@ func TestStopperRunAsyncTaskTracing(t *testing.T) { errC <- errors.Errorf("missing span") return } - sp = tr.StartSpan("child", tracing.WithParentAndAutoCollection(sp)) + sp = tr.StartSpan("child", tracing.WithParent(sp)) if sp.TraceID() == traceID { errC <- errors.Errorf("expected different trace") } diff --git a/pkg/util/tracing/bench_test.go b/pkg/util/tracing/bench_test.go index 62bef2bad0b9..051f71eb2f8b 100644 --- a/pkg/util/tracing/bench_test.go +++ b/pkg/util/tracing/bench_test.go @@ -52,10 +52,10 @@ func BenchmarkTracer_StartSpanCtx(b *testing.B) { WithForceRealSpan(), WithLogTags(&staticLogTags), }}, {"real,autoparent", []SpanOption{ - WithForceRealSpan(), WithParentAndAutoCollection(parSp), + WithForceRealSpan(), WithParent(parSp), }}, {"real,manualparent", []SpanOption{ - WithForceRealSpan(), WithParentAndManualCollection(parSp.Meta()), + WithForceRealSpan(), WithParent(parSp), WithDetachedRecording(), }}, } { b.Run(fmt.Sprintf("opts=%s", tc.name), func(b *testing.B) { @@ -91,7 +91,7 @@ func BenchmarkSpan_GetRecording(b *testing.B) { run(b, sp) }) - child := tr.StartSpan("bar", WithParentAndAutoCollection(sp), WithForceRealSpan()) + child := tr.StartSpan("bar", WithParent(sp), WithForceRealSpan()) b.Run("child-only", func(b *testing.B) { run(b, child) }) @@ -110,7 +110,7 @@ func BenchmarkRecordingWithStructuredEvent(b *testing.B) { for i := 0; i < b.N; i++ { root := tr.StartSpan("foo") root.RecordStructured(ev) - child := tr.StartSpan("bar", WithParentAndAutoCollection(root)) + child := tr.StartSpan("bar", WithParent(root)) child.RecordStructured(ev) child.Finish() _ = root.GetRecording(RecordingStructured) diff --git a/pkg/util/tracing/collector/collector_test.go b/pkg/util/tracing/collector/collector_test.go index 5224b0cc225f..ea9ebba6b804 100644 --- a/pkg/util/tracing/collector/collector_test.go +++ b/pkg/util/tracing/collector/collector_test.go @@ -72,20 +72,20 @@ func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, tracingpb.TraceID, time.Sleep(10 * time.Millisecond) // Start a child span on "node 1". - child := t1.StartSpan("root.child", tracing.WithParentAndAutoCollection(root)) + child := t1.StartSpan("root.child", tracing.WithParent(root)) // Sleep a bit so that everything that comes afterwards has higher timestamps // than the one we just assigned. Otherwise the sorting is not deterministic. time.Sleep(10 * time.Millisecond) // Start a remote child span on "node 2". - childRemoteChild := t2.StartSpan("root.child.remotechild", tracing.WithParentAndManualCollection(child.Meta())) + childRemoteChild := t2.StartSpan("root.child.remotechild", tracing.WithRemoteParent(child.Meta())) childRemoteChild.RecordStructured(newTestStructured("root.child.remotechild")) time.Sleep(10 * time.Millisecond) // Start another remote child span on "node 2" that we finish. - childRemoteChildFinished := t2.StartSpan("root.child.remotechilddone", tracing.WithParentAndManualCollection(child.Meta())) + childRemoteChildFinished := t2.StartSpan("root.child.remotechilddone", tracing.WithRemoteParent(child.Meta())) child.ImportRemoteSpans(childRemoteChildFinished.FinishAndGetRecording(tracing.RecordingVerbose)) // Start a root span on "node 2". @@ -93,14 +93,14 @@ func setupTraces(t1, t2 *tracing.Tracer) (tracingpb.TraceID, tracingpb.TraceID, root2.RecordStructured(newTestStructured("root2")) // Start a child span on "node 2". - child2 := t2.StartSpan("root2.child", tracing.WithParentAndAutoCollection(root2)) + child2 := t2.StartSpan("root2.child", tracing.WithParent(root2)) // Start a remote child span on "node 1". - child2RemoteChild := t1.StartSpan("root2.child.remotechild", tracing.WithParentAndManualCollection(child2.Meta())) + child2RemoteChild := t1.StartSpan("root2.child.remotechild", tracing.WithRemoteParent(child2.Meta())) time.Sleep(10 * time.Millisecond) // Start another remote child span on "node 1". - anotherChild2RemoteChild := t1.StartSpan("root2.child.remotechild2", tracing.WithParentAndManualCollection(child2.Meta())) + anotherChild2RemoteChild := t1.StartSpan("root2.child.remotechild2", tracing.WithRemoteParent(child2.Meta())) return root.TraceID(), root2.TraceID(), func() { for _, span := range []*tracing.Span{root, child, childRemoteChild, root2, child2, child2RemoteChild, anotherChild2RemoteChild} { diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index df494fd608d0..95a8c050c2a4 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -55,6 +55,14 @@ type crdbSpan struct { mu crdbSpanMu } +type childRef struct { + *crdbSpan + // collectRecording is set if this child's recording should be included in the + // parent's recording. This is usually the case, except for children created + // with the WithDetachedRecording() option. + collectRecording bool +} + type crdbSpanMu struct { syncutil.Mutex @@ -94,7 +102,7 @@ type crdbSpanMu struct { // the respective child moves to finishedChildren. // // The spans are not maintained in a particular order. - openChildren []*crdbSpan + openChildren []childRef // finishedChildren contains the recordings of finished children (and // grandchildren recursively). This includes remote child span recordings // that were manually imported, as well as recordings from local children @@ -166,7 +174,9 @@ func (s *crdbSpan) finish() bool { // Shallow-copy the children so they can be processed outside the lock. children = make([]*crdbSpan, len(s.mu.recording.openChildren)) - copy(children, s.mu.recording.openChildren) + for i, c := range s.mu.recording.openChildren { + children[i] = c.crdbSpan + } // We'll operate on the parent outside of the child's lock. parent = s.mu.parent @@ -223,11 +233,19 @@ func (s *crdbSpan) TraceID() tracingpb.TraceID { // GetRecording is part of the RegistrySpan interface. func (s *crdbSpan) GetRecording(recType RecordingType) Recording { + return s.getRecordingImpl(recType, false /* includeDetachedChildren */) +} + +func (s *crdbSpan) GetFullRecording(recType RecordingType) Recording { + return s.getRecordingImpl(recType, true /* includeDetachedChildren */) +} + +func (s *crdbSpan) getRecordingImpl(recType RecordingType, includeDetachedChildren bool) Recording { switch recType { case RecordingVerbose: - return s.getVerboseRecording() + return s.getVerboseRecording(includeDetachedChildren) case RecordingStructured: - return s.getStructuredRecording() + return s.getStructuredRecording(includeDetachedChildren) case RecordingOff: return nil default: @@ -236,7 +254,7 @@ func (s *crdbSpan) GetRecording(recType RecordingType) Recording { } // getVerboseRecording returns the Span's recording, including its children. -func (s *crdbSpan) getVerboseRecording() Recording { +func (s *crdbSpan) getVerboseRecording(includeDetachedChildren bool) Recording { if s == nil { return nil // noop span } @@ -249,7 +267,9 @@ func (s *crdbSpan) getVerboseRecording() Recording { result = append(result, s.mu.recording.finishedChildren...) for _, child := range s.mu.recording.openChildren { - result = append(result, child.getVerboseRecording()...) + if child.collectRecording || includeDetachedChildren { + result = append(result, child.getVerboseRecording(includeDetachedChildren)...) + } } s.mu.Unlock() @@ -271,7 +291,7 @@ func (s *crdbSpan) getVerboseRecording() Recording { // this span (even if the events had been recorded on different spans). // // The caller does not take ownership of the events. -func (s *crdbSpan) getStructuredRecording() Recording { +func (s *crdbSpan) getStructuredRecording(includeDetachedChildren bool) Recording { s.mu.Lock() defer s.mu.Unlock() buffer := make([]*tracingpb.StructuredRecord, 0, 3) @@ -281,7 +301,9 @@ func (s *crdbSpan) getStructuredRecording() Recording { } } for _, c := range s.mu.recording.openChildren { - buffer = c.getStructuredEventsRecursively(buffer) + if c.collectRecording || includeDetachedChildren { + buffer = c.getStructuredEventsRecursively(buffer, includeDetachedChildren) + } } if len(buffer) == 0 && s.mu.recording.structured.Len() == 0 { @@ -449,13 +471,15 @@ func (s *crdbSpan) recordInternalLocked(payload memorySizable, buffer *sizeLimit // getStructuredEventsRecursively returns the structured events accumulated by // this span and its finished and still-open children. func (s *crdbSpan) getStructuredEventsRecursively( - buffer []*tracingpb.StructuredRecord, + buffer []*tracingpb.StructuredRecord, includeDetachedChildren bool, ) []*tracingpb.StructuredRecord { s.mu.Lock() defer s.mu.Unlock() buffer = s.getStructuredEventsLocked(buffer) for _, c := range s.mu.recording.openChildren { - buffer = c.getStructuredEventsRecursively(buffer) + if c.collectRecording || includeDetachedChildren { + buffer = c.getStructuredEventsRecursively(buffer, includeDetachedChildren) + } } for _, c := range s.mu.recording.finishedChildren { for i := range c.StructuredRecords { @@ -563,7 +587,7 @@ func (s *crdbSpan) getRecordingNoChildrenLocked( return rs } -func (s *crdbSpan) addChild(child *crdbSpan) { +func (s *crdbSpan) addChild(child *crdbSpan, collectChildRec bool) { s.mu.Lock() defer s.mu.Unlock() if s.recordingType() == RecordingOff { @@ -574,7 +598,10 @@ func (s *crdbSpan) addChild(child *crdbSpan) { } if len(s.mu.recording.openChildren) < maxChildrenPerSpan { - s.mu.recording.openChildren = append(s.mu.recording.openChildren, child) + s.mu.recording.openChildren = append( + s.mu.recording.openChildren, + childRef{crdbSpan: child, collectRecording: collectChildRec}, + ) } } @@ -588,47 +615,58 @@ func (s *crdbSpan) addChild(child *crdbSpan) { // This is only called if the respective child had been linked to the parent - // i.e. only if the parent was recording when the child started. func (s *crdbSpan) childFinished(child *crdbSpan) { - // Collect the recording outside of s' lock, to avoid locking the parent and - // the child at the same time (to not have to think about deadlocks). + s.mu.Lock() + defer s.mu.Unlock() + var childIdx int + found := false + for i, c := range s.mu.recording.openChildren { + if c.crdbSpan == child { + childIdx = i + found = true + break + } + } + if !found { + panic("child not present in parent") + } + + collectChildRec := s.mu.recording.openChildren[childIdx].collectRecording + + // Unlink the child. + l := len(s.mu.recording.openChildren) + s.mu.recording.openChildren[childIdx] = s.mu.recording.openChildren[l-1] + s.mu.recording.openChildren[l-1].crdbSpan = nil // Make the child available to GC. + s.mu.recording.openChildren = s.mu.recording.openChildren[:l-1] + + // Collect the child's recording. + + if s.recordingType() == RecordingOff || !collectChildRec { + return + } + var rec Recording var events []*tracingpb.StructuredRecord var verbose bool - var structured bool switch s.recordingType() { case RecordingOff: + panic("should have been handled above") case RecordingVerbose: verbose = true rec = child.GetRecording(RecordingVerbose) case RecordingStructured: - structured = true events = make([]*tracingpb.StructuredRecord, 0, 3) - events = child.getStructuredEventsRecursively(events) + events = child.getStructuredEventsRecursively(events, false /* includeDetachedChildren */) default: panic(fmt.Sprintf("unrecognized recording mode: %v", s.recordingType())) } - s.mu.Lock() - defer s.mu.Unlock() - if verbose { s.recordFinishedChildrenLocked(rec) - } else if structured { + } else { for i := range events { s.recordInternalLocked(events[i], &s.mu.recording.structured) } } - - // Unlink the child. - l := len(s.mu.recording.openChildren) - for i, c := range s.mu.recording.openChildren { - if c != child { - continue - } - s.mu.recording.openChildren[i] = s.mu.recording.openChildren[l-1] - s.mu.recording.openChildren[l-1] = nil - s.mu.recording.openChildren = s.mu.recording.openChildren[:l-1] - break - } } // parentFinished makes s a root. diff --git a/pkg/util/tracing/doc.go b/pkg/util/tracing/doc.go index 1ec86ecaaaa2..0e5aa4c9fa65 100644 --- a/pkg/util/tracing/doc.go +++ b/pkg/util/tracing/doc.go @@ -94,7 +94,7 @@ // [6]: `crdbSpan` // [7]: `Span.SetVerbose`. To understand the specifics of what exactly is // captured in Span recording, when Spans have children that may be either -// local or remote, look towards `WithParentAnd{Auto,Manual}Collection` +// local or remote, look towards `WithParent` and `WithDetachedRecording`. // [8]: `Tracer.{InjectMetaInto,ExtractMetaFrom}` // [9]: `SpanMeta` // [10]: `{Client,Server}Interceptor` diff --git a/pkg/util/tracing/grpc_interceptor.go b/pkg/util/tracing/grpc_interceptor.go index 2c183b753655..284dd3398265 100644 --- a/pkg/util/tracing/grpc_interceptor.go +++ b/pkg/util/tracing/grpc_interceptor.go @@ -118,7 +118,7 @@ func ServerInterceptor(tracer *Tracer) grpc.UnaryServerInterceptor { ctx, serverSpan := tracer.StartSpanCtx( ctx, info.FullMethod, - WithParentAndManualCollection(spanMeta), + WithRemoteParent(spanMeta), WithServerSpanKind, ) defer serverSpan.Finish() @@ -161,7 +161,7 @@ func StreamServerInterceptor(tracer *Tracer) grpc.StreamServerInterceptor { ctx, serverSpan := tracer.StartSpanCtx( ss.Context(), info.FullMethod, - WithParentAndManualCollection(spanMeta), + WithRemoteParent(spanMeta), WithServerSpanKind, ) defer serverSpan.Finish() @@ -243,7 +243,7 @@ func ClientInterceptor(tracer *Tracer, init func(*Span)) grpc.UnaryClientInterce } clientSpan := tracer.StartSpan( method, - WithParentAndAutoCollection(parent), + WithParent(parent), WithClientSpanKind, ) init(clientSpan) @@ -292,7 +292,7 @@ func StreamClientInterceptor(tracer *Tracer, init func(*Span)) grpc.StreamClient clientSpan := tracer.StartSpan( method, - WithParentAndAutoCollection(parent), + WithParent(parent), WithClientSpanKind, ) init(clientSpan) diff --git a/pkg/util/tracing/grpc_interceptor_test.go b/pkg/util/tracing/grpc_interceptor_test.go index 8305248ea290..18cb189d80c2 100644 --- a/pkg/util/tracing/grpc_interceptor_test.go +++ b/pkg/util/tracing/grpc_interceptor_test.go @@ -227,7 +227,7 @@ func TestGRPCInterceptors(t *testing.T) { runtime.GC() testutils.SucceedsSoon(t, func() error { return tr.VisitSpans(func(sp tracing.RegistrySpan) error { - rec := sp.GetRecording(tracing.RecordingVerbose)[0] + rec := sp.GetFullRecording(tracing.RecordingVerbose)[0] return errors.Newf("leaked span: %s %s", rec.Operation, rec.Tags) }) }) diff --git a/pkg/util/tracing/service/service.go b/pkg/util/tracing/service/service.go index 80bf71c4cd29..61d314d011ac 100644 --- a/pkg/util/tracing/service/service.go +++ b/pkg/util/tracing/service/service.go @@ -53,7 +53,7 @@ func (s *Service) GetSpanRecordings( if span.TraceID() != request.TraceID { return nil } - recording := span.GetRecording(tracing.RecordingVerbose) + recording := span.GetFullRecording(tracing.RecordingVerbose) if recording != nil { resp.Recordings = append(resp.Recordings, tracingservicepb.GetSpanRecordingsResponse_Recording{RecordedSpans: recording}) diff --git a/pkg/util/tracing/service/service_test.go b/pkg/util/tracing/service/service_test.go index be72030a14d1..8ed99f20633d 100644 --- a/pkg/util/tracing/service/service_test.go +++ b/pkg/util/tracing/service/service_test.go @@ -30,21 +30,20 @@ func TestTracingServiceGetSpanRecordings(t *testing.T) { setupTraces := func() (tracingpb.TraceID, func()) { // Start a root span. root1 := tracer1.StartSpan("root1", tracing.WithRecording(tracing.RecordingVerbose)) - - child1 := tracer1.StartSpan("root1.child", tracing.WithParentAndAutoCollection(root1)) - - time.Sleep(10 * time.Millisecond) - + child1 := tracer1.StartSpan("root1.child", tracing.WithParent(root1)) + child2 := tracer1.StartSpan("root1.child.detached", tracing.WithParent(child1), tracing.WithDetachedRecording()) // Create a span that will be added to the tracers' active span map, but // will share the same traceID as root. - fork1 := tracer1.StartSpan("fork1", tracing.WithParentAndManualCollection(root1.Meta())) + child3 := tracer1.StartSpan("root1.remote_child", tracing.WithRemoteParent(child2.Meta())) + + time.Sleep(10 * time.Millisecond) // Start span with different trace ID. root2 := tracer1.StartSpan("root2", tracing.WithRecording(tracing.RecordingVerbose)) root2.Record("root2") return root1.TraceID(), func() { - for _, span := range []*tracing.Span{root1, child1, fork1, root2} { + for _, span := range []*tracing.Span{root1, child1, child2, child3, root2} { span.Finish() } } @@ -71,9 +70,11 @@ func TestTracingServiceGetSpanRecordings(t *testing.T) { tags: _unfinished=1 _verbose=1 span: root1.child tags: _unfinished=1 _verbose=1 + span: root1.child.detached + tags: _unfinished=1 _verbose=1 `)) require.NoError(t, tracing.CheckRecordedSpans(resp.Recordings[1].RecordedSpans, ` - span: fork1 + span: root1.remote_child tags: _unfinished=1 _verbose=1 `)) } diff --git a/pkg/util/tracing/span.go b/pkg/util/tracing/span.go index 5544818995e1..d72c20a62fd0 100644 --- a/pkg/util/tracing/span.go +++ b/pkg/util/tracing/span.go @@ -42,8 +42,8 @@ const ( // The CockroachDB-internal Span (crdbSpan) is more complex because // rather than reporting to some external sink, the caller's "owner" // must propagate the trace data back across process boundaries towards -// the root of the trace span tree; see WithParentAndAutoCollection -// and WithParentAndManualCollection, respectively. +// the root of the trace span tree; see WithParent +// and WithRemoteParent, respectively. // // Additionally, the internal span type also supports turning on, stopping, // and restarting its data collection (see Span.StartRecording), and this is @@ -131,8 +131,8 @@ func (sp *Span) ImportRemoteSpans(remoteSpans []tracingpb.RecordedSpan) { // Meta returns the information which needs to be propagated across process // boundaries in order to derive child spans from this Span. This may return -// nil, which is a valid input to `WithParentAndManualCollection`, if the Span -// has been optimized out. +// nil, which is a valid input to WithRemoteParent, if the Span has been +// optimized out. func (sp *Span) Meta() SpanMeta { // It shouldn't be done in practice, but it is allowed to call Meta on // a finished span. diff --git a/pkg/util/tracing/span_options.go b/pkg/util/tracing/span_options.go index 86e4db3aabc0..6df3f4418f5d 100644 --- a/pkg/util/tracing/span_options.go +++ b/pkg/util/tracing/span_options.go @@ -46,14 +46,30 @@ var followsFromAttribute = []attribute.KeyValue{attribute.String("follows-from", // field comment below are invoked as arguments to `Tracer.StartSpan`. // See the SpanOption interface for a synopsis. type spanOptions struct { - Parent *Span // see WithParentAndAutoCollection - RemoteParent SpanMeta // see WithParentAndManualCollection - RefType spanReferenceType // see WithFollowsFrom - LogTags *logtags.Buffer // see WithLogTags - Tags map[string]interface{} // see WithTags - ForceRealSpan bool // see WithForceRealSpan - SpanKind oteltrace.SpanKind // see WithSpanKind - Sterile bool // see WithSterile + Parent *Span // see WithParent + // ParentDoesNotCollectRecording is set by WithDetachedRecording. It means + // that, although this span has a parent, the parent should generally not + // include this child's recording when the parent is asked for its own + // recording. Usually, a parent span includes all its children in its + // recording. However, sometimes that's not desired; sometimes the creator of + // a child span has a different plan for how the recording of that child will + // end up being collected and reported to where it ultimately needs to go. + // Still, even in these cases, a parent-child relationship is still useful + // (for example for the purposes of the active spans registry), so the child + // span cannot simply be created as a root. + // + // For example, in the case of DistSQL, each processor in a flow has its own + // span, as a child of the flow. The DistSQL infrastructure organizes the + // collection of each processor span recording independently, without relying + // on collecting the recording of the flow's span. + ParentDoesNotCollectRecording bool + RemoteParent SpanMeta // see WithRemoteParent + RefType spanReferenceType // see WithFollowsFrom + LogTags *logtags.Buffer // see WithLogTags + Tags map[string]interface{} // see WithTags + ForceRealSpan bool // see WithForceRealSpan + SpanKind oteltrace.SpanKind // see WithSpanKind + Sterile bool // see WithSterile // recordingTypeExplicit is set if the WithRecording() option was used. In // that case, spanOptions.recordingType() returns recordingTypeOpt below. If @@ -111,23 +127,28 @@ func (opts *spanOptions) otelContext() (oteltrace.Span, oteltrace.SpanContext) { // SpanOption is the interface satisfied by options to `Tracer.StartSpan`. // A synopsis of the options follows. For details, see their comments. // -// - WithParentAndAutoCollection: create a child Span from a Span. -// - WithParentAndManualCollection: create a child Span from a SpanMeta. -// - WithFollowsFrom: indicate that child may outlive parent. +// - WithParent: create a child Span with a local parent. +// - WithRemoteParent: create a child Span with a remote parent. +// - WithFollowsFrom: hint that child may outlive parent. // - WithLogTags: populates the Span tags from a `logtags.Buffer`. // - WithCtxLogTags: like WithLogTags, but takes a `context.Context`. // - WithTags: adds tags to a Span on creation. // - WithForceRealSpan: prevents optimizations that can avoid creating a real span. +// - WithDetachedRecording: don't include the recording in the parent. type SpanOption interface { apply(spanOptions) spanOptions } -type parentAndAutoCollectionOption Span +type parentOption Span -// WithParentAndAutoCollection instructs StartSpan to create a child Span -// from a parent Span. +// WithParent instructs StartSpan to create a child Span from a (local) parent +// Span. // -// WithParentAndAutoCollection will be a no-op (i.e. the span resulting from +// In case when the parent span is created with a different Tracer (generally, +// when the parent lives in a different process), WithRemoteParent should be +// used. +// +// WithParent will be a no-op (i.e. the span resulting from // applying this option will be a root span, just as if this option hadn't been // specified) in the following cases: // - if `sp` is nil @@ -153,69 +174,97 @@ type parentAndAutoCollectionOption Span // which corresponds to the expectation that the parent span will // wait for the child to Finish(). If this expectation does not hold, // WithFollowsFrom should be added to the StartSpan invocation. -// -// When the parent Span is not available at the caller, -// WithParentAndManualCollection should be used, which incurs an -// obligation to manually propagate the trace data to the parent Span. -func WithParentAndAutoCollection(sp *Span) SpanOption { +func WithParent(sp *Span) SpanOption { if sp == nil || sp.IsNoop() || sp.IsSterile() { - return (*parentAndAutoCollectionOption)(nil) + return (*parentOption)(nil) } - return (*parentAndAutoCollectionOption)(sp) + return (*parentOption)(sp) } -func (p *parentAndAutoCollectionOption) apply(opts spanOptions) spanOptions { +func (p *parentOption) apply(opts spanOptions) spanOptions { opts.Parent = (*Span)(p) return opts } -type parentAndManualCollectionOption SpanMeta +type remoteParent SpanMeta -// WithParentAndManualCollection instructs StartSpan to create a -// child span descending from a parent described via a SpanMeta. In -// contrast with WithParentAndAutoCollection, the caller must call -// `Span.GetRecording` when finishing the returned Span, and propagate the -// result to the parent Span by calling `Span.ImportRemoteSpans` on it. +// WithRemoteParent instructs StartSpan to create a child span descending from a +// parent described via a SpanMeta. Generally this parent span lives in a +// different process. // -// The canonical use case for this is around RPC boundaries, where a -// server handling a request wants to create a child span descending -// from a parent on a remote machine. +// For the purposes of trace recordings, there's no mechanism ensuring that the +// child's recording will be passed to the parent span. When that's desired, it +// has to be done manually by calling Span.GetRecording() and propagating the +// result to the parent by calling Span.ImportRemoteSpans(). +// +// The canonical use case for this is around RPC boundaries, where a server +// handling a request wants to create a child span descending from a parent on a +// remote machine. // // node 1 (network) node 2 // -------------------------------------------------------------------------- // Span.Meta() ----------> sp2 := Tracer.StartSpan( -// WithParentAndManualCollection(.)) +// WithRemoteParent(.)) // doSomething(sp2) -// sp2.Finish() -// Span.ImportRemoteSpans(.) <---------- sp2.GetRecording() +// Span.ImportRemoteSpans(.) <---------- sp2.FinishAndGetRecording() // -// By default, the child span is derived using a ChildOf relationship, -// which corresponds to the expectation that the parent span will -// wait for the child to Finish(). If this expectation does not hold, -// WithFollowsFrom should be added to the StartSpan invocation. -func WithParentAndManualCollection(parent SpanMeta) SpanOption { +// By default, the child span is derived using a ChildOf relationship, which +// corresponds to the expectation that the parent span will usually wait for the +// child to Finish(). If this expectation does not hold, WithFollowsFrom should +// be added to the StartSpan invocation. +func WithRemoteParent(parent SpanMeta) SpanOption { if parent.sterile { - return parentAndManualCollectionOption{} + return remoteParent{} } - return (parentAndManualCollectionOption)(parent) + return (remoteParent)(parent) } -func (p parentAndManualCollectionOption) apply(opts spanOptions) spanOptions { +func (p remoteParent) apply(opts spanOptions) spanOptions { opts.RemoteParent = (SpanMeta)(p) return opts } +type detachedRecording struct{} + +var detachedRecordingSingleton = SpanOption(detachedRecording{}) + +func (o detachedRecording) apply(opts spanOptions) spanOptions { + opts.ParentDoesNotCollectRecording = true + return opts +} + +// WithDetachedRecording configures the span to not be included in the parent's +// recording (if any) under most circumstances. Usually, a parent span includes +// all its children in its recording. However, sometimes that's not desired; +// sometimes the creator of a child span has a different plan for how the +// recording of that child will end up being collected and reported to where it +// ultimately needs to go. Still, even in these cases, a parent-child +// relationship is still useful (for example for the purposes of the active +// spans registry), so the child span cannot simply be created as a root. +// +// For example, in the case of DistSQL, each processor in a flow has its own +// span, as a child of the flow. The DistSQL infrastructure organizes the +// collection of each processor span recording independently, without relying +// on collecting the recording of the flow's span. +// +// In the case when the parent's recording is collected through the span +// registry, this option is ignore since, in that case, we want as much info as +// possible. +func WithDetachedRecording() SpanOption { + return detachedRecordingSingleton +} + type followsFromOpt struct{} var followsFromSingleton = SpanOption(followsFromOpt{}) // WithFollowsFrom instructs StartSpan to link the child span to its parent // using a different kind of relationship than the regular parent-child one, -// should a child span be created (i.e. should WithParentAndAutoCollection or -// WithParentAndManualCollection be supplied as well). This relationship was +// should a child span be created (i.e. should WithParent or +// WithRemoteParent be supplied as well). This relationship was // called "follows-from" in the old OpenTracing API. This only matters if the // trace is sent to an OpenTelemetry tracer; CRDB itself ignores it (what -// matters for CRDB is the AutoCollection vs ManualCollection distinction). +// matters for CRDB is the WithDetachedTrace option). // OpenTelemetry does not have a concept of a follows-from relationship at the // moment; specifying this option results in the child having a Link to the // parent. diff --git a/pkg/util/tracing/span_test.go b/pkg/util/tracing/span_test.go index b5e4a503e05c..fa685ef29c51 100644 --- a/pkg/util/tracing/span_test.go +++ b/pkg/util/tracing/span_test.go @@ -65,7 +65,7 @@ func TestRecordingString(t *testing.T) { wireSpanMeta, err := tr2.ExtractMetaFrom(carrier) require.NoError(t, err) - remoteChild := tr2.StartSpan("remote child", WithParentAndManualCollection(wireSpanMeta)) + remoteChild := tr2.StartSpan("remote child", WithRemoteParent(wireSpanMeta), WithDetachedRecording()) root.Record("root 2") remoteChild.Record("remote child 1") @@ -74,7 +74,7 @@ func TestRecordingString(t *testing.T) { root.Record("root 3") - ch2 := tr.StartSpan("local child", WithParentAndAutoCollection(root)) + ch2 := tr.StartSpan("local child", WithParent(root)) root.Record("root 4") ch2.Record("local child 1") ch2.Finish() @@ -154,11 +154,11 @@ func TestRecordingInRecording(t *testing.T) { tr := NewTracer() root := tr.StartSpan("root", WithRecording(RecordingVerbose)) - child := tr.StartSpan("child", WithParentAndAutoCollection(root), WithRecording(RecordingVerbose)) + child := tr.StartSpan("child", WithParent(root), WithRecording(RecordingVerbose)) // The remote grandchild is also recording, however since it's remote the spans // have to be imported into the parent manually (this would usually happen via // code at the RPC boundaries). - grandChild := tr.StartSpan("grandchild", WithParentAndManualCollection(child.Meta())) + grandChild := tr.StartSpan("grandchild", WithParent(child), WithDetachedRecording()) child.ImportRemoteSpans(grandChild.FinishAndGetRecording(RecordingVerbose)) childRec := child.FinishAndGetRecording(RecordingVerbose) require.NoError(t, CheckRecordedSpans(childRec, ` @@ -191,7 +191,7 @@ func TestImportRemoteSpans(t *testing.T) { t.Run(fmt.Sprintf("%s=%t", "verbose-child=", verbose), func(t *testing.T) { tr := NewTracerWithOpt(context.Background()) sp := tr.StartSpan("root", WithRecording(RecordingStructured)) - ch := tr.StartSpan("child", WithParentAndManualCollection(sp.Meta())) + ch := tr.StartSpan("child", WithParent(sp), WithDetachedRecording()) ch.RecordStructured(&types.Int32Value{Value: 4}) if verbose { sp.SetVerbose(true) @@ -332,11 +332,11 @@ func TestChildSpanRegisteredWithRecordingParent(t *testing.T) { tr := NewTracer() sp := tr.StartSpan("root", WithRecording(RecordingStructured)) defer sp.Finish() - ch := tr.StartSpan("child", WithParentAndAutoCollection(sp)) + ch := tr.StartSpan("child", WithParent(sp)) defer ch.Finish() children := sp.i.crdb.mu.recording.openChildren require.Len(t, children, 1) - require.Equal(t, ch.i.crdb, children[0]) + require.Equal(t, ch.i.crdb, children[0].crdbSpan) ch.RecordStructured(&types.Int32Value{Value: 5}) // Check that the child's structured event is in the recording. rec := sp.GetRecording(RecordingStructured) @@ -351,7 +351,7 @@ func TestSpanMaxChildren(t *testing.T) { sp := tr.StartSpan("root", WithRecording(RecordingStructured)) defer sp.Finish() for i := 0; i < maxChildrenPerSpan+123; i++ { - tr.StartSpan(fmt.Sprintf("child %d", i), WithParentAndAutoCollection(sp)) + tr.StartSpan(fmt.Sprintf("child %d", i), WithParent(sp)) exp := i + 1 if exp > maxChildrenPerSpan { exp = maxChildrenPerSpan @@ -475,8 +475,8 @@ func TestStructureRecording(t *testing.T) { t.Run(fmt.Sprintf("finish2=%t", finishCh2), func(t *testing.T) { tr := NewTracerWithOpt(context.Background(), WithTestingKnobs(TracerTestingKnobs{ForceRealSpans: true})) sp := tr.StartSpan("root", WithRecording(RecordingStructured)) - ch1 := tr.StartSpan("child", WithParentAndAutoCollection(sp)) - ch2 := tr.StartSpan("grandchild", WithParentAndAutoCollection(ch1)) + ch1 := tr.StartSpan("child", WithParent(sp)) + ch2 := tr.StartSpan("grandchild", WithParent(ch1)) for i := int32(0); i < 5; i++ { sp.RecordStructured(&types.Int32Value{Value: i}) ch1.RecordStructured(&types.Int32Value{Value: i}) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 3ab719552002..79109ebd373f 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -608,12 +608,12 @@ func (t *Tracer) startSpanGeneric( if opts.Parent.IsNoop() { // This method relies on the parent, if any, not being a no-op. A no-op // parent should have been optimized away by the - // WithParentAndAutoCollection option. + // WithParent option. panic("invalid no-op parent") } if opts.Parent.IsSterile() { // A sterile parent should have been optimized away by - // WithParentAndAutoCollection. + // WithParent. panic("invalid sterile parent") } if opts.Parent.Tracer() != t { @@ -701,7 +701,7 @@ func (t *Tracer) startSpanGeneric( octx optimizedContext // Pre-allocated buffers for the span. tagsAlloc [3]attribute.KeyValue - childrenAlloc [4]*crdbSpan + childrenAlloc [4]childRef structuredEventsAlloc [3]interface{} }{} @@ -742,7 +742,7 @@ func (t *Tracer) startSpanGeneric( if opts.Parent != nil && opts.Parent.i.crdb != nil { if opts.Parent.i.crdb.recordingType() != RecordingOff { s.i.crdb.mu.parent = opts.Parent.i.crdb - opts.Parent.i.crdb.addChild(s.i.crdb) + opts.Parent.i.crdb.addChild(s.i.crdb, !opts.ParentDoesNotCollectRecording) } } s.i.crdb.enableRecording(opts.recordingType()) @@ -937,8 +937,13 @@ type RegistrySpan interface { // TraceID returns an identifier for the trace that this span is part of. TraceID() tracingpb.TraceID - // GetRecording returns the recording of the trace rooted at this span. - GetRecording(recType RecordingType) Recording + // GetFullRecording returns the recording of the trace rooted at this span. + // + // This includes the recording of child spans created with the + // WithDetachedRecording option. In other situations, the recording of such + // children is not included in the parent's recording but, in the case of the + // span registry, we want as much information as possible to be included. + GetFullRecording(recType RecordingType) Recording // SetVerbose sets the verbosity of the span appropriately and // recurses on its children. @@ -1007,13 +1012,14 @@ func ForkSpan(ctx context.Context, opName string) (context.Context, *Span) { if sp == nil { return ctx, nil } - collectionOpt := WithParentAndManualCollection(sp.Meta()) + opts := make([]SpanOption, 0, 3) if sp.Tracer().ShouldRecordAsyncSpans() { - // Using auto collection here ensures that recordings from async spans - // also show up at the parent. - collectionOpt = WithParentAndAutoCollection(sp) + opts = append(opts, WithParent(sp)) + } else { + opts = append(opts, WithParent(sp), WithDetachedRecording()) } - return sp.Tracer().StartSpanCtx(ctx, opName, WithFollowsFrom(), collectionOpt) + opts = append(opts, WithFollowsFrom()) + return sp.Tracer().StartSpanCtx(ctx, opName, opts...) } // EnsureForkSpan is like ForkSpan except that, if there is no span in ctx, it @@ -1024,12 +1030,12 @@ func EnsureForkSpan(ctx context.Context, tr *Tracer, opName string) (context.Con // If there's a span in ctx, we use it as a parent. if sp != nil { tr = sp.Tracer() - if !tr.ShouldRecordAsyncSpans() { - opts = append(opts, WithParentAndManualCollection(sp.Meta())) + if tr.ShouldRecordAsyncSpans() { + opts = append(opts, WithParent(sp)) } else { // Using auto collection here ensures that recordings from async spans // also show up at the parent. - opts = append(opts, WithParentAndAutoCollection(sp)) + opts = append(opts, WithParent(sp), WithDetachedRecording()) } opts = append(opts, WithFollowsFrom()) } @@ -1048,23 +1054,11 @@ func ChildSpan(ctx context.Context, opName string) (context.Context, *Span) { if sp == nil { return ctx, nil } - return sp.Tracer().StartSpanCtx(ctx, opName, WithParentAndAutoCollection(sp)) -} - -// ChildSpanRemote is like ChildSpan but the new Span is created using -// WithParentAndManualCollection instead of WithParentAndAutoCollection. When -// this is used, it's the caller's duty to collect this span's recording and -// return it to the root span of the trace. -func ChildSpanRemote(ctx context.Context, opName string) (context.Context, *Span) { - sp := SpanFromContext(ctx) - if sp == nil { - return ctx, nil - } - return sp.Tracer().StartSpanCtx(ctx, opName, WithParentAndManualCollection(sp.Meta())) + return sp.Tracer().StartSpanCtx(ctx, opName, WithParent(sp)) } // EnsureChildSpan looks at the supplied Context. If it contains a Span, returns -// a child span via the WithParentAndAutoCollection option; otherwise starts a +// a child span via the WithParent option; otherwise starts a // new Span. In both cases, a context wrapping the Span is returned along with // the newly created Span. // @@ -1073,7 +1067,7 @@ func EnsureChildSpan( ctx context.Context, tr *Tracer, name string, os ...SpanOption, ) (context.Context, *Span) { slp := optsPool.Get().(*[]SpanOption) - *slp = append(*slp, WithParentAndAutoCollection(SpanFromContext(ctx))) + *slp = append(*slp, WithParent(SpanFromContext(ctx))) *slp = append(*slp, os...) ctx, sp := tr.StartSpanCtx(ctx, name, *slp...) // Clear and zero-length the slice. Note that we have to clear diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index 3d8ffa8fbc04..272a426f8b93 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -35,10 +35,10 @@ func TestStartSpanAlwaysTrace(t *testing.T) { require.True(t, tr.AlwaysTrace()) nilMeta := tr.noopSpan.Meta() require.True(t, nilMeta.Empty()) - sp := tr.StartSpan("foo", WithParentAndManualCollection(nilMeta)) + sp := tr.StartSpan("foo", WithRemoteParent(nilMeta)) require.False(t, sp.IsVerbose()) // parent was not verbose, so neither is sp require.False(t, sp.IsNoop()) - sp = tr.StartSpan("foo", WithParentAndAutoCollection(tr.noopSpan)) + sp = tr.StartSpan("foo", WithParent(tr.noopSpan)) require.False(t, sp.IsVerbose()) // parent was not verbose require.False(t, sp.IsNoop()) } @@ -55,7 +55,7 @@ func TestTracerRecording(t *testing.T) { // Noop span returns empty recording. require.Equal(t, Recording(nil), noop1.GetRecording(RecordingVerbose)) - noop2 := tr.StartSpan("noop2", WithParentAndManualCollection(noop1.Meta())) + noop2 := tr.StartSpan("noop2", WithParent(noop1), WithDetachedRecording()) if !noop2.IsNoop() { t.Error("expected noop child Span") } @@ -91,7 +91,7 @@ func TestTracerRecording(t *testing.T) { s1.SetVerbose(false) // Real parent --> real child. - real3 := tr.StartSpan("noop3", WithParentAndManualCollection(s1.Meta())) + real3 := tr.StartSpan("noop3", WithRemoteParent(s1.Meta())) if real3.IsNoop() { t.Error("expected real child Span") } @@ -100,7 +100,7 @@ func TestTracerRecording(t *testing.T) { s1.Recordf("x=%d", 1) s1.SetVerbose(true) s1.Recordf("x=%d", 2) - s2 := tr.StartSpan("b", WithParentAndAutoCollection(s1)) + s2 := tr.StartSpan("b", WithParent(s1)) if !s2.IsVerbose() { t.Error("recording Span should be verbose") } @@ -125,7 +125,7 @@ func TestTracerRecording(t *testing.T) { t.Fatal(err) } - s3 := tr.StartSpan("c", WithParentAndAutoCollection(s2)) + s3 := tr.StartSpan("c", WithParent(s2)) s3.Recordf("x=%d", 4) s3.SetTag("tag", attribute.StringValue("val")) @@ -172,7 +172,7 @@ func TestTracerRecording(t *testing.T) { func TestStartChildSpan(t *testing.T) { tr := NewTracer() sp1 := tr.StartSpan("parent", WithRecording(RecordingVerbose)) - sp2 := tr.StartSpan("child", WithParentAndAutoCollection(sp1)) + sp2 := tr.StartSpan("child", WithParent(sp1)) sp2.Finish() if err := CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` @@ -185,7 +185,7 @@ func TestStartChildSpan(t *testing.T) { } sp1 = tr.StartSpan("parent", WithRecording(RecordingVerbose)) - sp2 = tr.StartSpan("child", WithParentAndManualCollection(sp1.Meta())) + sp2 = tr.StartSpan("child", WithParent(sp1), WithDetachedRecording()) if err := CheckRecordedSpans(sp2.FinishAndGetRecording(RecordingVerbose), ` span: child tags: _verbose=1 @@ -200,7 +200,7 @@ func TestStartChildSpan(t *testing.T) { } sp1 = tr.StartSpan("parent", WithRecording(RecordingVerbose)) - sp2 = tr.StartSpan("child", WithParentAndAutoCollection(sp1), + sp2 = tr.StartSpan("child", WithParent(sp1), WithLogTags(logtags.SingleTagBuffer("key", "val"))) sp2.Finish() if err := CheckRecordedSpans(sp1.FinishAndGetRecording(RecordingVerbose), ` @@ -220,12 +220,12 @@ func TestSterileSpan(t *testing.T) { // Make the span verbose so that we can use its recording below to assert that // there were no children. sp1 := tr.StartSpan("parent", WithSterile(), WithRecording(RecordingVerbose)) - sp2 := tr.StartSpan("child", WithParentAndAutoCollection(sp1)) + sp2 := tr.StartSpan("child", WithParent(sp1)) require.Zero(t, sp2.i.crdb.parentSpanID) require.True(t, sp1.Meta().sterile) require.False(t, sp2.Meta().sterile) - sp3 := tr.StartSpan("child", WithParentAndManualCollection(sp1.Meta())) + sp3 := tr.StartSpan("child", WithParent(sp1), WithDetachedRecording()) require.Zero(t, sp3.i.crdb.parentSpanID) sp2.Finish() @@ -264,7 +264,7 @@ func TestTracerInjectExtract(t *testing.T) { if !wireSpanMeta.Empty() { t.Errorf("expected no-op span meta: %v", wireSpanMeta) } - noop2 := tr2.StartSpan("remote op", WithParentAndManualCollection(wireSpanMeta)) + noop2 := tr2.StartSpan("remote op", WithRemoteParent(wireSpanMeta)) if !noop2.IsNoop() { t.Fatalf("expected noop Span: %+v", noop2) } @@ -283,7 +283,7 @@ func TestTracerInjectExtract(t *testing.T) { if err != nil { t.Fatal(err) } - s2 := tr2.StartSpan("remote op", WithParentAndManualCollection(wireSpanMeta)) + s2 := tr2.StartSpan("remote op", WithRemoteParent(wireSpanMeta)) // Compare TraceIDs trace1 := s1.Meta().traceID @@ -338,7 +338,7 @@ func TestTracer_PropagateNonRecordingRealSpanAcrossRPCBoundaries(t *testing.T) { meta, err := tr2.ExtractMetaFrom(carrier) require.NoError(t, err) require.True(t, spanInclusionFuncForServer(tr2, meta)) - sp2 := tr2.StartSpan("tr2.child", WithParentAndManualCollection(meta)) + sp2 := tr2.StartSpan("tr2.child", WithRemoteParent(meta)) defer sp2.Finish() require.NotZero(t, sp2.i.crdb.spanID) } @@ -370,7 +370,7 @@ func TestOtelTracer(t *testing.T) { t.Fatal(err) } - tr.StartSpan("child", WithParentAndManualCollection(wireSpanMeta)) + tr.StartSpan("child", WithRemoteParent(wireSpanMeta)) rs := sr.Started() require.Len(t, rs, 2) @@ -400,10 +400,10 @@ func TestActiveSpanVisitorErrors(t *testing.T) { root := tr.StartSpan("root", WithForceRealSpan()) defer root.Finish() - child := tr.StartSpan("root.child", WithParentAndAutoCollection(root)) + child := tr.StartSpan("root.child", WithParent(root)) defer child.Finish() - remoteChild := tr.StartSpan("root.remotechild", WithParentAndManualCollection(child.Meta())) + remoteChild := tr.StartSpan("root.remotechild", WithParent(child), WithDetachedRecording()) defer remoteChild.Finish() var numVisited int @@ -426,7 +426,7 @@ func getSpanOpsWithFinished(t *testing.T, tr *Tracer) map[string]bool { spanOpsWithFinished := make(map[string]bool) require.NoError(t, tr.VisitSpans(func(sp RegistrySpan) error { - for _, rec := range sp.GetRecording(RecordingVerbose) { + for _, rec := range sp.GetFullRecording(RecordingVerbose) { spanOpsWithFinished[rec.Operation] = rec.Finished } return nil @@ -443,7 +443,7 @@ func getSortedSpanOps(t *testing.T, tr *Tracer) []string { var spanOps []string require.NoError(t, tr.VisitSpans(func(sp RegistrySpan) error { - for _, rec := range sp.GetRecording(RecordingVerbose) { + for _, rec := range sp.GetFullRecording(RecordingVerbose) { spanOps = append(spanOps, rec.Operation) } return nil @@ -460,11 +460,11 @@ func TestTracer_VisitSpans(t *testing.T) { tr2 := NewTracer() root := tr1.StartSpan("root", WithRecording(RecordingVerbose)) - child := tr1.StartSpan("root.child", WithParentAndAutoCollection(root)) + child := tr1.StartSpan("root.child", WithParent(root)) require.Len(t, tr1.activeSpansRegistry.mu.m, 1) - childChild := tr2.StartSpan("root.child.remotechild", WithParentAndManualCollection(child.Meta())) - childChildFinished := tr2.StartSpan("root.child.remotechilddone", WithParentAndManualCollection(child.Meta())) + childChild := tr2.StartSpan("root.child.remotechild", WithRemoteParent(child.Meta())) + childChildFinished := tr2.StartSpan("root.child.remotechilddone", WithRemoteParent(child.Meta())) require.Len(t, tr2.activeSpansRegistry.mu.m, 2) child.ImportRemoteSpans(childChildFinished.GetRecording(RecordingVerbose)) @@ -495,11 +495,11 @@ func TestSpanRecordingFinished(t *testing.T) { tr1 := NewTracer() root := tr1.StartSpan("root", WithRecording(RecordingVerbose)) - child := tr1.StartSpan("root.child", WithParentAndAutoCollection(root)) - childChild := tr1.StartSpan("root.child.child", WithParentAndAutoCollection(child)) + child := tr1.StartSpan("root.child", WithParent(root)) + childChild := tr1.StartSpan("root.child.child", WithParent(child)) tr2 := NewTracer() - remoteChildChild := tr2.StartSpan("root.child.remotechild", WithParentAndManualCollection(child.Meta())) + remoteChildChild := tr2.StartSpan("root.child.remotechild", WithRemoteParent(child.Meta())) child.ImportRemoteSpans(remoteChildChild.GetRecording(RecordingVerbose)) remoteChildChild.Finish() @@ -560,7 +560,7 @@ func TestSpanWithNoopParentIsInActiveSpans(t *testing.T) { tr := NewTracer() noop := tr.StartSpan("noop") require.True(t, noop.IsNoop()) - root := tr.StartSpan("foo", WithParentAndAutoCollection(noop), WithForceRealSpan()) + root := tr.StartSpan("foo", WithParent(noop), WithForceRealSpan()) require.Len(t, tr.activeSpansRegistry.mu.m, 1) visitor := func(sp RegistrySpan) error { require.Equal(t, root.i.crdb, sp) @@ -580,7 +580,7 @@ func TestConcurrentChildAndRecording(t *testing.T) { defer wg.Done() sp := tr.StartSpan( "child", - WithParentAndAutoCollection(rootSp), // links sp to rootSp + WithParent(rootSp), // links sp to rootSp WithSpanKind(oteltrace.SpanKindConsumer)) // causes a tag to be set sp.Finish() }() @@ -595,8 +595,8 @@ func TestConcurrentChildAndRecording(t *testing.T) { func TestFinishedSpanInRecording(t *testing.T) { tr := NewTracer() s1 := tr.StartSpan("a", WithRecording(RecordingVerbose)) - s2 := tr.StartSpan("b", WithParentAndAutoCollection(s1)) - s3 := tr.StartSpan("c", WithParentAndAutoCollection(s2)) + s2 := tr.StartSpan("b", WithParent(s1)) + s3 := tr.StartSpan("c", WithParent(s2)) // Check that s2 is included in the recording both before and after it's // finished. @@ -629,8 +629,8 @@ span: a // Now the same thing, but finish s2 first. s1 = tr.StartSpan("a", WithRecording(RecordingVerbose)) - s2 = tr.StartSpan("b", WithParentAndAutoCollection(s1)) - tr.StartSpan("c", WithParentAndAutoCollection(s2)) + s2 = tr.StartSpan("b", WithParent(s1)) + tr.StartSpan("c", WithParent(s2)) s2.Finish() require.NoError(t, CheckRecordedSpans(s1.FinishAndGetRecording(RecordingVerbose), ` @@ -651,8 +651,8 @@ func TestRegistryOrphanSpansBecomeRoots(t *testing.T) { // s1 must be recording because, otherwise, the child spans are not linked to // it. s1 := tr.StartSpan("parent", WithRecording(RecordingStructured)) - s2 := tr.StartSpan("child1", WithParentAndAutoCollection(s1)) - s3 := tr.StartSpan("child2", WithParentAndAutoCollection(s1)) + s2 := tr.StartSpan("child1", WithParent(s1)) + s3 := tr.StartSpan("child2", WithParent(s1)) require.Equal(t, []*crdbSpan{s1.i.crdb}, tr.activeSpansRegistry.testingAll()) s1.Finish() require.ElementsMatch(t, []*crdbSpan{s2.i.crdb, s3.i.crdb}, tr.activeSpansRegistry.testingAll())