Skip to content

Commit

Permalink
Merge #58123
Browse files Browse the repository at this point in the history
58123: tracing: optimize context.Context allocations r=knz a=tbg

Since in our practical usage of tracing each created `Span` is certain to
be wrapped in a `Context`, it makes sense to allocate that wrapping
Context alongside the Span.

This does not immediately change the allocations in my sanity check

```
go test -benchtime 1000x -benchmem -memprofilerate 1 -memprofile ctx.prof \
        -bench 'BenchmarkScan/MultinodeCockroach/count=1$/limit=1$' \
        ./pkg/bench/
[...] 809 allocs/op
```

because the current hot path in `startSpanGeneric` is that in which the
noop span is returned, and that path does not benefit from the
optimization.

However, with `trace.mode = background`, we get ~850 alloc/ops with this
commit, and ~884 allocs/op without. (There is certainly a fair bit of noise in
this, so don't walk away from this thinking that we've saved 34 allocs - the
message is "it helps in that case").

Additionally, the newly added microbenchmark shows that StartSpanCtx clocks in
at a single allocation (when called without options that require additional
allocations).

More optimizations are possible and will be driven by experiments. I expect
that these will primarily target the allocation overhead of constructing
SpanOptions, and the implementation of WithTags.

Release note: None


Co-authored-by: Tobias Schottdorf <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
3 people committed Dec 21, 2020
2 parents ff612f1 + a325ad2 commit 638c03a
Show file tree
Hide file tree
Showing 4 changed files with 196 additions and 55 deletions.
3 changes: 3 additions & 0 deletions pkg/util/tracing/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_library(
srcs = [
"annotate.go",
"annotate_nocgo.go",
"context.go",
"grpc_interceptor.go",
"recording.go",
"shadow.go",
Expand Down Expand Up @@ -47,13 +48,15 @@ go_library(
go_test(
name = "tracing_test",
srcs = [
"alloc_test.go",
"helpers_test.go",
"span_test.go",
"tags_test.go",
"tracer_test.go",
],
embed = [":tracing"],
deps = [
"//pkg/settings",
"//pkg/util/tracing/tracingpb",
"//vendor/github.com/cockroachdb/errors",
"//vendor/github.com/cockroachdb/logtags",
Expand Down
100 changes: 100 additions & 0 deletions pkg/util/tracing/alloc_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package tracing

import (
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/logtags"
"github.com/opentracing/opentracing-go"
)

// BenchmarkTracer_StartSpanCtx primarily helps keep
// tab on the allocation counts for starting a Span.
//
// This benchmark explicitly excludes construction of
// the SpanOptions, which require allocations as well.
func BenchmarkTracer_StartSpanCtx(b *testing.B) {
ctx := context.Background()

tr := NewTracer()
sv := settings.Values{}
tracingMode.Override(&sv, int64(modeBackground))
tr.Configure(&sv)

staticLogTags := logtags.Buffer{}
staticLogTags.Add("foo", "bar")

staticTag := opentracing.Tag{
Key: "statictag",
Value: "staticvalue",
}
b.ResetTimer()

parSp := tr.StartSpan("one-off", WithForceRealSpan())
defer parSp.Finish()

for _, tc := range []struct {
name string
opts []SpanOption
}{
{"none", nil},
{"logtag", []SpanOption{WithLogTags(&staticLogTags)}},
{"tag", []SpanOption{WithTags(staticTag)}},
{"autoparent", []SpanOption{WithParentAndAutoCollection(parSp)}},
{"manualparent", []SpanOption{WithParentAndManualCollection(parSp.Meta())}},
} {
b.Run(fmt.Sprintf("opts=%s", tc.name), func(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
newCtx, sp := tr.StartSpanCtx(ctx, "benching", tc.opts...)
_ = newCtx
_ = sp
}
})
}

}

// BenchmarkSpan_GetRecording microbenchmarks GetRecording
// when background tracing is enabled.
func BenchmarkSpan_GetRecording(b *testing.B) {
var sv settings.Values
tracingMode.Override(&sv, int64(modeBackground))
tr := NewTracer()
tr.Configure(&sv)

sp := tr.StartSpan("foo", WithForceRealSpan())

run := func(b *testing.B, sp *Span) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = sp.GetRecording()
}
}

b.ResetTimer()
b.Run("root-only", func(b *testing.B) {
run(b, sp)
})

child := tr.StartSpan("bar", WithParentAndAutoCollection(sp), WithForceRealSpan())
b.Run("child-only", func(b *testing.B) {
run(b, child)
})

b.Run("root-child", func(b *testing.B) {
run(b, sp)
})
}
87 changes: 87 additions & 0 deletions pkg/util/tracing/context.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package tracing

import "context"

type activeSpanKey struct{}

// noCtx is a singleton that we use internally to unify code paths that only
// optionally take a Context. The specific construction here does not matter,
// the only thing we need is that no outside caller could ever pass this
// context in (i.e. we can't use context.Background() and the like).
var noCtx context.Context = &struct{ context.Context }{context.Background()}

// SpanFromContext returns the *Span contained in the Context, if any.
func SpanFromContext(ctx context.Context) *Span {
val := ctx.Value(activeSpanKey{})
if sp, ok := val.(*Span); ok {
return sp
}
return nil
}

// optimizedContext is an implementation of context.Context special
// cased to carry a Span under activeSpanKey{}. By making an explicit
// type we unlock optimizations that save allocations by allocating
// the optimizedContext together with the Span it eventually carries.
type optimizedContext struct {
context.Context
sp *Span
}

func (ctx *optimizedContext) Value(k interface{}) interface{} {
if k == (interface{}(activeSpanKey{})) {
return ctx.sp
}
return ctx.Context.Value(k)
}

// maybeWrapCtx returns a Context wrapping the Span, with two exceptions:
// 1. if ctx==noCtx, it's a noop
// 2. if ctx contains the noop Span, and sp is also the noop Span, elide
// allocating a new Context.
//
// If a non-nil octx is passed in, it forms the returned Context. This can
// avoid allocations if the caller is able to allocate octx together with
// the Span, as is commonly possible when StartSpanCtx is used.
func maybeWrapCtx(ctx context.Context, octx *optimizedContext, sp *Span) (context.Context, *Span) {
if ctx == noCtx {
return noCtx, sp
}
// NB: we check sp != nil explicitly because some callers want to remove a
// Span from a Context, and thus pass nil.
if sp != nil && sp.isNoop() {
// If the context originally had the noop span, and we would now be wrapping
// the noop span in it again, we don't have to wrap at all and can save an
// allocation.
//
// Note that applying this optimization for a nontrivial ctxSp would
// constitute a bug: A real, non-recording span might later start recording.
// Besides, the caller expects to get their own span, and will .Finish() it,
// leading to an extra, premature call to Finish().
if ctxSp := SpanFromContext(ctx); ctxSp != nil && ctxSp.isNoop() {
return ctx, sp
}
}
if octx != nil {
octx.Context = ctx
octx.sp = sp
return octx, sp
}
return context.WithValue(ctx, activeSpanKey{}, sp), sp
}

// ContextWithSpan returns a Context wrapping the supplied Span.
func ContextWithSpan(ctx context.Context, sp *Span) context.Context {
ctx, _ = maybeWrapCtx(ctx, nil /* octx */, sp)
return ctx
}
61 changes: 6 additions & 55 deletions pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,13 +204,6 @@ func (t *Tracer) getShadowTracer() *shadowTracer {
return (*shadowTracer)(atomic.LoadPointer(&t.shadowTracer))
}

// noCtx is a singleton that we use internally to unify code paths that only
// optionally take a Context. The specific construction here does not matter,
// the only thing we need is that no outside caller could ever pass this
// context in (i.e. we can't use context.Background() and the like), and that
// we can compare it directly.
var noCtx context.Context = &struct{ context.Context }{context.Background()}

// StartSpan starts a Span. See SpanOption for details.
func (t *Tracer) StartSpan(operationName string, os ...SpanOption) *Span {
_, sp := t.StartSpanCtx(noCtx, operationName, os...)
Expand Down Expand Up @@ -247,32 +240,6 @@ func (t *Tracer) AlwaysTrace() bool {
return t.useNetTrace() || shadowTracer != nil
}

// maybeWrapCtx returns a Context wrapping the Span, with two exceptions:
// 1. if ctx==noCtx, it's a noop
// 2. if ctx contains the noop Span, and sp is also the noop Span, elide
// allocating a new Context.
func maybeWrapCtx(ctx context.Context, sp *Span) (context.Context, *Span) {
if ctx == noCtx {
return noCtx, sp
}
// NB: we check sp != nil explicitly because some callers want to remove a
// Span from a Context, and thus pass nil.
if sp != nil && sp.isNoop() {
// If the context originally had the noop span, and we would now be wrapping
// the noop span in it again, we don't have to wrap at all and can save an
// allocation.
//
// Note that applying this optimization for a nontrivial ctxSp would
// constitute a bug: A real, non-recording span might later start recording.
// Besides, the caller expects to get their own span, and will .Finish() it,
// leading to an extra, premature call to Finish().
if ctxSp := SpanFromContext(ctx); ctxSp != nil && ctxSp.isNoop() {
return ctx, sp
}
}
return context.WithValue(ctx, activeSpanKey{}, sp), sp
}

// startSpanGeneric is the implementation of StartSpanCtx and StartSpan. In
// the latter case, ctx == noCtx and the returned Context is the supplied one;
// otherwise the returned Context reflects the returned Span.
Expand Down Expand Up @@ -305,7 +272,7 @@ func (t *Tracer) startSpanGeneric(
if !t.AlwaysTrace() &&
opts.recordingType() == RecordingOff &&
!opts.ForceRealSpan {
return maybeWrapCtx(ctx, t.noopSpan)
return maybeWrapCtx(ctx, nil /* octx */, t.noopSpan)
}

if opts.LogTags == nil && opts.Parent != nil && !opts.Parent.isNoop() {
Expand Down Expand Up @@ -381,8 +348,9 @@ func (t *Tracer) startSpanGeneric(
// that *only* contains `ot` or `netTr`. This is just an artifact
// of the history of this code and may change in the future.
helper := struct {
Span Span
span Span
crdbSpan crdbSpan
octx optimizedContext
}{}

helper.crdbSpan = crdbSpan{
Expand All @@ -396,14 +364,14 @@ func (t *Tracer) startSpanGeneric(
duration: -1, // unfinished
},
}
helper.Span = Span{
helper.span = Span{
tracer: t,
crdb: &helper.crdbSpan,
ot: ot,
netTr: netTr,
}

s := &helper.Span
s := &helper.span

// Start recording if necessary. We inherit the recording type of the local parent, if any,
// over the remote parent, if any. If neither are specified, we're not recording.
Expand Down Expand Up @@ -442,7 +410,7 @@ func (t *Tracer) startSpanGeneric(
}
}

return maybeWrapCtx(ctx, s)
return maybeWrapCtx(ctx, &helper.octx, s)
}

type textMapWriterFn func(key, val string)
Expand Down Expand Up @@ -684,23 +652,6 @@ var optsPool = sync.Pool{
},
}

type activeSpanKey struct{}

// SpanFromContext returns the *Span contained in the Context, if any.
func SpanFromContext(ctx context.Context) *Span {
val := ctx.Value(activeSpanKey{})
if sp, ok := val.(*Span); ok {
return sp
}
return nil
}

// ContextWithSpan returns a Context wrapping the supplied Span.
func ContextWithSpan(ctx context.Context, sp *Span) context.Context {
ctx, _ = maybeWrapCtx(ctx, sp)
return ctx
}

// StartVerboseTrace takes in a context and returns a derived one with a
// Span in it that is recording verbosely. The caller takes ownership of
// this Span from the returned context and is in charge of Finish()ing it.
Expand Down

0 comments on commit 638c03a

Please sign in to comment.