From fc5c933fb6bc783096f258f668a518e08b0afca0 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Thu, 18 Nov 2021 15:02:23 -0500 Subject: [PATCH 1/2] tracing: assert against mixing tracers Creating a child with a different Tracer than the parent is broken because the child span can leak: if the parent is Finish()ed first, the child gets inserted into the parent's registry: https://github.com/cockroachdb/cockroach/blob/d60e17a7047bb16b6aed1585bc5d274bd748d04e/pkg/util/tracing/crdbspan.go#L186 However, when the child gets Finish()ed, we attempt to remove it from its Tracer's registry. Mixing Tracers like this seems fundamentally a bad idea, because it is inherently unclear what the behavior with respect to the registries should be. This patch adds an assertion at span creation time checking that the two Tracers involved are one and the same. Release note: None --- pkg/util/tracing/tracer.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 4240c6f9410b..3ab719552002 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -616,6 +616,16 @@ func (t *Tracer) startSpanGeneric( // WithParentAndAutoCollection. panic("invalid sterile parent") } + if opts.Parent.Tracer() != t { + // Creating a child with a different Tracer than the parent is not allowed + // because it would become unclear which active span registry the new span + // should belong to. In particular, the child could end up in the parent's + // registry if the parent Finish()es before the child, and then it would + // be leaked because Finish()ing the child would attempt to remove the + // span from the child tracer's registry. + panic(fmt.Sprintf("attempting to start span with parent from different Tracer. parent: %s, child: %s", + opts.Parent.OperationName(), opName)) + } } // Are we tracing everything, or have a parent, or want a real span, or were From 37175f77bf374d1bcb76bc39a65149788be06134 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Sat, 13 Nov 2021 13:44:49 -0500 Subject: [PATCH 2/2] tracing: untangle local/remove parent vs recording collection Before this patch, two concepts were bundled together: whether or not the parent of a span is local or remote to the child's node, and whether or not the parent should include the child in its recording. You could create a child span with either: - the WithParentAndAutoCollection option, which would give you a local parent which includes the child in its recording, or - the WithParentAndManualCollection option, which would give you a remote parent (which naturally would not / couldn't include the child in its recording). WithParentAndManualCollection is sometimes used even when the parent is local, which is a) quite confusing, because at a lower level it produced what was called a "remote parent" and b) sub-optimal, because the child looked like a "local root", which meant it had to be put into the active spans registry. Were it not for the bundling of the two concerns together, such a child of a local parent, but for whom the parent is not in charge of collecting the recording, would not need to be put directly into the span registry; such a span (like other child spans) should be part of the registry indirectly, through its parent - which is cheaper because it doesn't add contention on the global mutex lock. The funky case of a local parent whose recording does not include the child's is used by DistSQL processors - each processor gets a span whose recording is collected by the DistSQL infrastructure, independent of the parent. This patch untangles the parent-child relationship from the recording collection concern, and cleans up the API in the process. Now you have the following options for creating a span: WithParent(parent) - creates a parent/child relationship. The parent has a reference to the child. The child is part of the active spans registry through the parent. Unless WithDetachedRecording() is also specified, the parent's recording includes the child's. WithRemoteParent(parentMeta) - creates a parent/child relationship across process boundaries. WithDetachedRecording() - asks the parent to not include the new child in its recording. This is a no-op in case WithRemoteParent() is used. This option is used by DistSQL. So, we get a cleaner API, and a more efficient active spans registry implementation because the DistSQL spans are no longer directly part of it. We also get a new feature: given a span id, the trace registry is now able to collect the recording of all the children of that span, including the ones created with WithDetachedRecording. This is nice, since it'll make inspection of DistSQL traces easier (since they'll look more like regular traces, rather then a collection of unrelated root spans that all need to be looked at in isolation). WithDetachedRecording() is used in two cases (nothing really new here): - for DistSQL processors, like discussed - for async tasks which (may) outlive the parent. In particular, the stopper uses it for its FollowsFromSpan task option. This use of detached recording is debatable. It's not necessary for correctness, since there's no particular harm in having the async task be a regular child. It arguably produces better looking recordings, since they don't include info on the async task. It can also serve as an optimization (although currently it doesn't) - if we know that the async span is likely to outlive the parent, then we might be better off inserting it directly into the registry, as opposed to first inserting it into the parent, and only moving it to the registry when the parent Finish()es. Release note: None --- .../kvserver/replica_application_decoder.go | 6 +- pkg/kv/kvserver/replica_range_lease.go | 2 +- .../migrationmanager/manager_external_test.go | 16 +- pkg/sql/crdb_internal.go | 2 +- pkg/sql/crdb_internal_test.go | 33 ++-- pkg/sql/distsql/server.go | 4 +- pkg/sql/execinfra/processorsbase.go | 7 +- pkg/sql/sem/builtins/generator_builtins.go | 2 +- pkg/sql/tests/tracing_sql_test.go | 6 +- pkg/testutils/testcluster/testcluster.go | 2 +- pkg/util/stop/stopper_test.go | 2 +- pkg/util/tracing/bench_test.go | 8 +- pkg/util/tracing/collector/collector_test.go | 12 +- pkg/util/tracing/crdbspan.go | 104 +++++++++---- pkg/util/tracing/doc.go | 2 +- pkg/util/tracing/grpc_interceptor.go | 8 +- pkg/util/tracing/grpc_interceptor_test.go | 2 +- pkg/util/tracing/service/service.go | 2 +- pkg/util/tracing/service/service_test.go | 17 ++- pkg/util/tracing/span.go | 8 +- pkg/util/tracing/span_options.go | 141 ++++++++++++------ pkg/util/tracing/span_test.go | 20 +-- pkg/util/tracing/tracer.go | 52 +++---- pkg/util/tracing/tracer_test.go | 66 ++++---- 24 files changed, 304 insertions(+), 220 deletions(-) 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())