-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
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
Showing
4 changed files
with
196 additions
and
55 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
}) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters