Skip to content

Commit

Permalink
Merge #135146
Browse files Browse the repository at this point in the history
135146: sql,tracing: reduce overhead of tracing r=tbg a=tbg

The below is almost the result - it also included #135567, which likely needs a bit more discussion, as well as #135682 which merged separately.

```
$ env N=15 T=1000x PKG=./pkg/sql/tests BENCHES=BenchmarkSysbench/SQL/3node/oltp_read_write scripts/bench 3d083fe tracing-opt

                                                       old   new
Sysbench/SQL/3node/oltp_read_write-8           5.922m ± 2%   5.629m ± 4%   -4.94% (p=0.004 n=15)
Sysbench/SQL/3node/oltp_read_write-8          2.544Mi ± 0%   2.399Mi ± 1%  -5.70% (p=0.000 n=15)
Sysbench/SQL/3node/oltp_read_write-8           18.98k ± 0%   17.46k ± 1%   -7.96% (p=0.000 n=15)
```

<details>Alloc profiles<summary>Details</summary>
<p>

The alloc profiles below were generated with:

```
rm -f *.pb.gz; ./dev bench --ignore-cache --stream-output --bench-mem ./pkg/sql/tests --filter BenchmarkSysbench/SQL/3node/oltp_read_write --test-args '-test.memprofile=after.pb.gz -test.benchtime=10s -test.cpu=8' -- --test_env 'COCKROACH_TESTING_MEM_PROFILES=/Users/tbg/crdb'

pprof -http :6060 -lines -base BenchmarkSysbench_SQL_3node_oltp_read_write.allocs.pre.pb.gz after.pb.gz
```

The above prevents setup-related allocations from showing up in the allocation profile.

Note that with `sql.txn_stats.sample_rate=0`, the `tracing.` related allocs disappear entirely. We could
consider reducing the sample rate by another factor of 10 to `.1%` (from 1%) or to make it more adaptive.
There is likely little benefit to sampling very frequent fingerprints at 1%.

![after](https://github.com/user-attachments/assets/869ed6c8-5771-49b8-b653-5c05b6cb4892)
![before](https://github.com/user-attachments/assets/038dab4e-956a-4600-bae2-f44e34bd4e3e)

</p>
</details> 

Epic: CRDB-43584


Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Nov 20, 2024
2 parents 9e5ffb1 + 85b6c92 commit 8d089dd
Show file tree
Hide file tree
Showing 11 changed files with 50 additions and 41 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -1193,7 +1193,7 @@ func (ds *DistSender) Send(
}

ctx = ds.AnnotateCtx(ctx)
ctx, sp := tracing.EnsureChildSpan(ctx, ds.AmbientContext.Tracer, "dist sender send")
ctx, sp := tracing.ChildSpan(ctx, "dist sender send")
defer sp.Finish()

// Send proxy requests directly through the transport. Any errors are
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_application_decoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ func (d *replicaDecoder) createTracingSpans(ctx context.Context) {
propCtx := ctx // raft scheduler's ctx
var propSp *tracing.Span
// If the client has a trace, put a child into propCtx.
if sp := tracing.SpanFromContext(cmd.proposal.Context()); sp != nil {
if sp := tracing.SpanFromContext(cmd.proposal.Context()); sp != nil && !sp.IsNoop() {
propCtx, propSp = sp.Tracer().StartSpanCtx(
propCtx, "local proposal", tracing.WithParent(sp),
)
Expand Down
8 changes: 4 additions & 4 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -1945,10 +1945,10 @@ func setupSpanForIncomingRPC(
remoteParent := !ba.TraceInfo.Empty()
if !remoteParent {
// This is either a local request which circumvented gRPC, or a remote
// request that didn't specify tracing information. In the former case,
// EnsureChildSpan will create a child span, in the former case we'll get a
// root span.
ctx, newSpan = tracing.EnsureChildSpan(ctx, tr, grpcinterceptor.BatchMethodName, tracing.WithServerSpanKind)
// request that didn't specify tracing information. We make a child span
// if the incoming request would like to be traced.
ctx, newSpan = tracing.ChildSpan(ctx,
grpcinterceptor.BatchMethodName, tracing.WithServerSpanKind)
} else {
// Non-local call. Tracing information comes from the request proto.

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,7 @@ func (ex *connExecutor) execStmtInOpenState(
ast := parserStmt.AST
var sp *tracing.Span
if !isPausablePortal() || !portal.pauseInfo.execStmtInOpenState.cleanup.isComplete {
ctx, sp = tracing.EnsureChildSpan(ctx, ex.server.cfg.AmbientCtx.Tracer, "sql query")
ctx, sp = tracing.ChildSpan(ctx, "sql query")
// TODO(andrei): Consider adding the placeholders as tags too.
sp.SetTag("statement", attribute.StringValue(parserStmt.SQL))
ctx = withStatement(ctx, ast)
Expand Down Expand Up @@ -1553,7 +1553,7 @@ func (ex *connExecutor) reportSessionDataChanges(fn func() error) error {
}

func (ex *connExecutor) commitSQLTransactionInternal(ctx context.Context) (retErr error) {
ctx, sp := tracing.EnsureChildSpan(ctx, ex.server.cfg.AmbientCtx.Tracer, "commit sql txn")
ctx, sp := tracing.ChildSpan(ctx, "commit sql txn")
defer sp.Finish()

defer func() {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func (ex *connExecutor) execPrepare(
return ev, payload
}

ctx, sp := tracing.EnsureChildSpan(ctx, ex.server.cfg.AmbientCtx.Tracer, "prepare stmt")
ctx, sp := tracing.ChildSpan(ctx, "prepare stmt")
defer sp.Finish()

// The anonymous statement can be overwritten.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/execinfra/processorsbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -840,7 +840,7 @@ func ProcessorSpan(
eventListeners ...tracing.EventListener,
) (context.Context, *tracing.Span) {
sp := tracing.SpanFromContext(ctx)
if sp == nil {
if sp == nil || sp.IsNoop() {
return ctx, nil
}
var listenersOpt tracing.SpanOption
Expand Down
11 changes: 5 additions & 6 deletions pkg/util/log/ambient_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,12 +130,11 @@ func (ac *AmbientContext) annotateCtxInternal(ctx context.Context) context.Conte
}

// AnnotateCtxWithSpan annotates the given context with the information in
// AmbientContext (see AnnotateCtx) and opens a span.
// AmbientContext (see AnnotateCtx).
//
// If the given context has a span, the new span is a child of that span.
// Otherwise, the Tracer in AmbientContext is used to create a new root span.
//
// The caller is responsible for closing the span (via Span.Finish).
// If the given context has a trace span, a child span is created. The returned
// span may be nil, but either way the caller is responsible for eventually
// closing the span (via Span.Finish, which is valid on the nil Span).
func (ac *AmbientContext) AnnotateCtxWithSpan(
ctx context.Context, opName string,
) (context.Context, *tracing.Span) {
Expand All @@ -155,7 +154,7 @@ func (ac *AmbientContext) AnnotateCtxWithSpan(
}
}

return tracing.EnsureChildSpan(ctx, ac.Tracer, opName)
return tracing.ChildSpan(ctx, opName)
}

// MakeTestingAmbientContext creates an AmbientContext for use in tests,
Expand Down
7 changes: 2 additions & 5 deletions pkg/util/log/ambient_context_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,16 +65,13 @@ func TestAnnotateCtxSpan(t *testing.T) {
t.Fatal(err)
}

// Annotate a context that has no span. The tracer will create a non-recordable
// span. We just check here that AnnotateCtxWithSpan properly returns it to the
// caller.
// Annotate a context that has no span. The tracer will not create a span.

ac.Tracer = tracer
ctx, sp := ac.AnnotateCtxWithSpan(context.Background(), "s")
defer sp.Finish()
require.Nil(t, sp)
require.Equal(t, sp, tracing.SpanFromContext(ctx))
require.NotNil(t, sp)
require.False(t, sp.IsVerbose())
}

func TestAnnotateCtxNodeStoreReplica(t *testing.T) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/util/stop/stopper.go
Original file line number Diff line number Diff line change
Expand Up @@ -474,9 +474,9 @@ func (s *Stopper) RunAsyncTaskEx(ctx context.Context, opt TaskOpts, f func(conte
var sp *tracing.Span
switch opt.SpanOpt {
case FollowsFromSpan:
ctx, sp = tracing.EnsureForkSpan(ctx, s.tracer, opt.TaskName)
ctx, sp = tracing.ForkSpan(ctx, opt.TaskName)
case ChildSpan:
ctx, sp = tracing.EnsureChildSpan(ctx, s.tracer, opt.TaskName)
ctx, sp = tracing.ChildSpan(ctx, opt.TaskName)
case SterileRootSpan:
ctx, sp = s.tracer.StartSpanCtx(ctx, opt.TaskName, tracing.WithSterile())
default:
Expand Down
32 changes: 20 additions & 12 deletions pkg/util/stop/stopper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -704,19 +704,27 @@ func TestStopperRunAsyncTaskTracing(t *testing.T) {
event: async 2`))
}

// Test that RunAsyncTask creates root spans when the caller doesn't have a
// Test that RunAsyncTask creates root spans only if the caller has a
// span.
func TestStopperRunAsyncTaskCreatesRootSpans(t *testing.T) {
defer leaktest.AfterTest(t)()
tr := tracing.NewTracer()
ctx := context.Background()
s := stop.NewStopper(stop.WithTracer(tr))
defer s.Stop(ctx)
c := make(chan *tracing.Span)
require.NoError(t, s.RunAsyncTask(ctx, "test",
func(ctx context.Context) {
c <- tracing.SpanFromContext(ctx)
},
))
require.NotNil(t, <-c)

testutils.RunTrueAndFalse(t, "hasSpan", func(t *testing.T, hasSpan bool) {
tr := tracing.NewTracer()
ctx := context.Background()
s := stop.NewStopper(stop.WithTracer(tr))
defer s.Stop(ctx)
c := make(chan *tracing.Span)
if hasSpan {
var sp *tracing.Span
ctx, sp = tr.StartSpanCtx(ctx, "root", tracing.WithForceRealSpan())
defer sp.Finish()
}
require.NoError(t, s.RunAsyncTask(ctx, "test",
func(ctx context.Context) {
c <- tracing.SpanFromContext(ctx)
},
))
require.Equal(t, hasSpan, <-c != nil)
})
}
17 changes: 11 additions & 6 deletions pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -1592,19 +1592,24 @@ func EnsureForkSpan(ctx context.Context, tr *Tracer, opName string) (context.Con
return tr.StartSpanCtx(ctx, opName, opts...)
}

// ChildSpan creates a child span of the current one, if any. Recordings from
// child spans are automatically propagated to the parent span, and the tags are
// inherited from the context's log tags automatically. Also see `ForkSpan`,
// for the other kind of derived span relation.
// ChildSpan creates a child span of the current one, if any, via the WithParent
// option. Recordings from child spans are automatically propagated to the
// parent span, and the tags are inherited from the context's log tags
// automatically. Also see `ForkSpan`, for the other kind of derived span
// relation.
//
// A context wrapping the newly created span is returned, along with the span
// itself. If non-nil, the caller is responsible for eventually Finish()ing it.
func ChildSpan(ctx context.Context, opName string) (context.Context, *Span) {
func ChildSpan(ctx context.Context, opName string, os ...SpanOption) (context.Context, *Span) {
sp := SpanFromContext(ctx)
if sp == nil {
return ctx, nil
}
return sp.Tracer().StartSpanCtx(ctx, opName, WithParent(sp))
if len(os) == 0 {
return sp.Tracer().StartSpanCtx(ctx, opName, WithParent(sp))
}
os = append(os[:len(os):len(os)], WithParent(sp))
return sp.Tracer().StartSpanCtx(ctx, opName, os...)
}

// EnsureChildSpan looks at the supplied Context. If it contains a Span, returns
Expand Down

0 comments on commit 8d089dd

Please sign in to comment.