diff --git a/pkg/base/serverident/BUILD.bazel b/pkg/base/serverident/BUILD.bazel index c9c429131dec..5180b55d762e 100644 --- a/pkg/base/serverident/BUILD.bazel +++ b/pkg/base/serverident/BUILD.bazel @@ -5,4 +5,5 @@ go_library( srcs = ["server_ident.go"], importpath = "github.com/cockroachdb/cockroach/pkg/base/serverident", visibility = ["//visibility:public"], + deps = ["//pkg/util/ctxutil"], ) diff --git a/pkg/base/serverident/server_ident.go b/pkg/base/serverident/server_ident.go index 1a0c6a3424d7..4a3544fd0f3d 100644 --- a/pkg/base/serverident/server_ident.go +++ b/pkg/base/serverident/server_ident.go @@ -5,16 +5,20 @@ package serverident -import "context" +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" +) // SystemTenantID is the string representation of // roachpb.SystemTenantID. Injected at initialization to avoid // an import dependency cycle. See SetSystemTenantID. var SystemTenantID string -// serverIdentificationContextKey is the type of a context.Value key -// used to carry ServerIdentificationPayload values. -type serverIdentificationContextKey struct{} +// ServerIdentificationContextKey is the fast value key used to annotate a +// context with a ServerIdentificationPayload. +var ServerIdentificationContextKey = ctxutil.RegisterFastValueKey() // ContextWithServerIdentification returns a context annotated with the provided // server identity. Use ServerIdentificationFromContext(ctx) to retrieve it from @@ -22,16 +26,18 @@ type serverIdentificationContextKey struct{} func ContextWithServerIdentification( ctx context.Context, serverID ServerIdentificationPayload, ) context.Context { - return context.WithValue(ctx, serverIdentificationContextKey{}, serverID) + return ctxutil.WithFastValue(ctx, ServerIdentificationContextKey, serverID) } // ServerIdentificationFromContext retrieves the server identity put in the // context by ContextWithServerIdentification. func ServerIdentificationFromContext(ctx context.Context) ServerIdentificationPayload { - r := ctx.Value(serverIdentificationContextKey{}) + r := ctxutil.FastValue(ctx, ServerIdentificationContextKey) if r == nil { return nil } + // TODO(radu): an interface-to-interface conversion is not great in a hot + // path. Maybe the type should be just a func instead of an interface. return r.(ServerIdentificationPayload) } @@ -57,7 +63,7 @@ const ( IdentifyInstanceID // IdentifyTenantID retrieves the tenant ID of the server. IdentifyTenantID - // IdentifyTenantLabel retrieves the tenant name of the server. + // IdentifyTenantName retrieves the tenant name of the server. IdentifyTenantName ) diff --git a/pkg/ccl/sqlproxyccl/BUILD.bazel b/pkg/ccl/sqlproxyccl/BUILD.bazel index 0f19bea9122b..e1f6e9c1d867 100644 --- a/pkg/ccl/sqlproxyccl/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgwirebase", "//pkg/util/cache", + "//pkg/util/ctxutil", "//pkg/util/grpcutil", "//pkg/util/httputil", "//pkg/util/log", diff --git a/pkg/ccl/sqlproxyccl/server.go b/pkg/ccl/sqlproxyccl/server.go index 06e4ac7dd300..d552aa93568b 100644 --- a/pkg/ccl/sqlproxyccl/server.go +++ b/pkg/ccl/sqlproxyccl/server.go @@ -14,6 +14,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/util/cache" + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/httputil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -453,20 +454,20 @@ func (s *Server) shouldLogError( return limiter.ShouldLog() } -// requestTagsContextKey is the type of a context.Value key used to carry the -// request tags map in a context.Context object. -type requestTagsContextKey struct{} +// requestTagsContextKey is the fast value key used to carry the request tags +// map in a context.Context object. +var requestTagsContextKey = ctxutil.RegisterFastValueKey() // contextWithRequestTags returns a context annotated with the provided request // tags map. Use requestTagsFromContext(ctx) to retrieve it back. func contextWithRequestTags(ctx context.Context, reqTags map[string]interface{}) context.Context { - return context.WithValue(ctx, requestTagsContextKey{}, reqTags) + return ctxutil.WithFastValue(ctx, requestTagsContextKey, reqTags) } // requestTagsFromContext retrieves the request tags map stored in the context // via contextWithRequestTags. func requestTagsFromContext(ctx context.Context) map[string]interface{} { - r := ctx.Value(requestTagsContextKey{}) + r := ctxutil.FastValue(ctx, requestTagsContextKey) if r == nil { return nil } diff --git a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel index d6793fc4f920..627e9c427c28 100644 --- a/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel +++ b/pkg/kv/kvserver/kvflowcontrol/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/util/admission/admissionpb", + "//pkg/util/ctxutil", "@com_github_cockroachdb_redact//:redact", "@com_github_dustin_go_humanize//:go-humanize", ], diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go index 8dcc4e63ad3e..3cff8a763ba0 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" "github.com/cockroachdb/redact" "github.com/dustin/go-humanize" ) @@ -436,7 +437,7 @@ func (s Stream) SafeFormat(p redact.SafePrinter, verb rune) { p.Printf("t%s/s%s", tenantSt, s.StoreID.String()) } -type raftAdmissionMetaKey struct{} +var RaftAdmissionMetaKey = ctxutil.RegisterFastValueKey() // ContextWithMeta returns a Context wrapping the supplied raft admission meta, // if any. @@ -445,7 +446,7 @@ type raftAdmissionMetaKey struct{} // #104154. func ContextWithMeta(ctx context.Context, meta *kvflowcontrolpb.RaftAdmissionMeta) context.Context { if meta != nil { - ctx = context.WithValue(ctx, raftAdmissionMetaKey{}, meta) + ctx = ctxutil.WithFastValue(ctx, RaftAdmissionMetaKey, meta) } return ctx } @@ -453,7 +454,7 @@ func ContextWithMeta(ctx context.Context, meta *kvflowcontrolpb.RaftAdmissionMet // MetaFromContext returns the raft admission meta embedded in the Context, if // any. func MetaFromContext(ctx context.Context) *kvflowcontrolpb.RaftAdmissionMeta { - val := ctx.Value(raftAdmissionMetaKey{}) + val := ctxutil.FastValue(ctx, RaftAdmissionMetaKey) h, ok := val.(*kvflowcontrolpb.RaftAdmissionMeta) if !ok { return nil diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index f4b2c66b4ea2..e30d48d7bd6e 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -547,6 +547,7 @@ go_library( "//pkg/util/collatedstring", "//pkg/util/ctxgroup", "//pkg/util/ctxlog", + "//pkg/util/ctxutil", "//pkg/util/duration", "//pkg/util/encoding", "//pkg/util/encoding/csv", diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 813a19fef26e..6550e42e42ef 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -70,6 +70,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/ctxlog" + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/fsm" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -4682,38 +4683,34 @@ func (ps connExPrepStmtsAccessor) DeleteAll(ctx context.Context) { ) } -// contextStatementKey is an empty type for the handle associated with the -// statement value (see context.Value). -type contextStatementKey struct{} +var contextStatementKey = ctxutil.RegisterFastValueKey() // withStatement adds a SQL statement to the provided context. The statement // will then be included in crash reports which use that context. func withStatement(ctx context.Context, stmt tree.Statement) context.Context { - return context.WithValue(ctx, contextStatementKey{}, stmt) + return ctxutil.WithFastValue(ctx, contextStatementKey, stmt) } // statementFromCtx returns the statement value from a context, or nil if unset. func statementFromCtx(ctx context.Context) tree.Statement { - stmt := ctx.Value(contextStatementKey{}) + stmt := ctxutil.FastValue(ctx, contextStatementKey) if stmt == nil { return nil } return stmt.(tree.Statement) } -// contextGistKey is an empty type for the handle associated with the -// gist value (see context.Value). -type contextPlanGistKey struct{} +var contextPlanGistKey = ctxutil.RegisterFastValueKey() func withPlanGist(ctx context.Context, gist string) context.Context { if gist == "" { return ctx } - return context.WithValue(ctx, contextPlanGistKey{}, gist) + return ctxutil.WithFastValue(ctx, contextPlanGistKey, gist) } func planGistFromCtx(ctx context.Context) string { - val := ctx.Value(contextPlanGistKey{}) + val := ctxutil.FastValue(ctx, contextPlanGistKey) if val != nil { return val.(string) } diff --git a/pkg/util/ctxutil/BUILD.bazel b/pkg/util/ctxutil/BUILD.bazel index 339a42d3079a..bb1aba0f7a6f 100644 --- a/pkg/util/ctxutil/BUILD.bazel +++ b/pkg/util/ctxutil/BUILD.bazel @@ -7,19 +7,23 @@ go_library( "canceler_1_21_bazel.go", "context.go", "doc.go", + "fast_value.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/util/ctxutil", visibility = ["//visibility:public"], deps = [ "//pkg/util/buildutil", - "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_logtags//:logtags", ], ) go_test( name = "ctxutil_test", - srcs = ["context_test.go"], + srcs = [ + "context_test.go", + "fast_value_test.go", + ], embed = [":ctxutil"], deps = [ "//pkg/util/leaktest", diff --git a/pkg/util/ctxutil/context.go b/pkg/util/ctxutil/context.go index 63eaf3a2daf7..405aa98a7fa4 100644 --- a/pkg/util/ctxutil/context.go +++ b/pkg/util/ctxutil/context.go @@ -7,10 +7,10 @@ package ctxutil import ( "context" + "fmt" _ "unsafe" // Must import unsafe to enable linkname. "github.com/cockroachdb/cockroach/pkg/util/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/log" ) // WhenDoneFunc is the callback invoked by context when it becomes done. @@ -49,7 +49,7 @@ func WhenDone(parent context.Context, done WhenDoneFunc) bool { // But, be safe and loudly fail tests in case somebody introduces strange // context implementation. if buildutil.CrdbTestBuild && !CanDirectlyDetectCancellation(parent) { - log.Fatalf(parent, "expected context that supports direct cancellation detection, found %T", parent) + panic(fmt.Sprintf("expected context that supports direct cancellation detection, found %T", parent)) } propagateCancel(parent, done) diff --git a/pkg/util/ctxutil/fast_value.go b/pkg/util/ctxutil/fast_value.go new file mode 100644 index 000000000000..e9de71d21371 --- /dev/null +++ b/pkg/util/ctxutil/fast_value.go @@ -0,0 +1,177 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package ctxutil + +import ( + "context" + "sync/atomic" + + "github.com/cockroachdb/logtags" +) + +// FastValue retrieves the value for the given key. +func FastValue(ctx context.Context, key FastValueKey) any { + // This case is not necessary, but it should improve the fast path a bit. + if c, ok := ctx.(*fastValuesCtx); ok { + return c.values[key] + } + if v := ctx.Value(fastValuesAncestorKey{}); v != nil { + return v.(*fastValuesCtx).values[key] + } + return nil +} + +// WithFastValue creates a new context with a new value for the given key. It +// overrides any existing value for that same key. +// +// The given value will be returned by FastValue(ctx, key). The key must have +// been generated using RegisterFastValueKey(). +// +// This is a more efficient alternative to using context.WithValue() and +// Context.Value(). +func WithFastValue(parent context.Context, key FastValueKey, val any) context.Context { + //c := fastValuesAlloc.Alloc() + //c.Context = parent + c := &fastValuesCtx{Context: parent} + if p, ok := parent.(*fastValuesCtx); ok { + // Connect to the grandparent context, since the parent won't do anything. + c.Context = p.Context + c.values = p.values + } else if v := parent.Value(fastValuesAncestorKey{}); v != nil { + c.values = v.(*fastValuesCtx).values + } + c.values[key] = val + return c +} + +// WithFastValues creates a new context that with new values for multiple keys. +// Any values set in fv override any existing values for those keys. +// +// Sample usage: +// +// var fv FastValues +// fv.Set(key1, val1) +// fv.Set(key2, val2) +// ctx = WithFastValues(ctx, &fv) +func WithFastValues(parent context.Context) FastValuesBuilder { + var bld FastValuesBuilder + bld.parent = parent + if p, ok := parent.(*fastValuesCtx); ok { + // Connect to the grandparent context, since the parent won't do anything. + bld.parent = p.Context + bld.values = p.values + } else if v := parent.Value(fastValuesAncestorKey{}); v != nil { + bld.values = v.(*fastValuesCtx).values + } + return bld +} + +// FastValuesBuilder contains multiple values; used for WithFastValues. +type FastValuesBuilder struct { + parent context.Context + values [MaxFastValues]any + changed bool +} + +// Assert that we can't overflow the bitmap. +const _ uint32 = 1 << (MaxFastValues - 1) + +// Get gets the value for the key in the context being built. If this key was +// set before, it returns the last value passed to Set(); otherwise it returns +// the value in the context that was passed to WithFastValues(). +func (b *FastValuesBuilder) Get(key FastValueKey) any { + return b.values[key] +} + +// Set sets the value for the key in the context being built. +func (b *FastValuesBuilder) Set(key FastValueKey, val any) { + b.changed = true + b.values[key] = val +} + +func (b *FastValuesBuilder) Finish() context.Context { + if !b.changed { + return b.parent + } + //c := fastValuesAlloc.Alloc() + //c.Context = parent + //c.values = b.values + c := &fastValuesCtx{ + Context: b.parent, + values: b.values, + } + b.parent = nil + return c +} + +// RegisterFastValueKey creates a key that can be used with WithFastValue(). This +// is intended to be called from global initialization code. +// +// Only MaxFastValues calls to RegisterFastValueKey are allowed for the lifetime +// of the binary; only a handful of very frequent in-context values should use +// this infrastructure. +func RegisterFastValueKey() FastValueKey { + n := numFastValues.Add(1) + if n > MaxFastValues { + panic("too many fast values registered") + } + return FastValueKey(n - 1) +} + +// FastValueKey is a key that can be used to get and set a fast value. +type FastValueKey uint8 + +// MaxFastValues is the number of FastValueKeys that can be registered. +const MaxFastValues = 8 + +var numFastValues atomic.Uint32 + +type fastValuesCtx struct { + context.Context + values [MaxFastValues]any +} + +//var fastValuesAlloc = cralloc.MakeBatchAllocator[fastValuesCtx]() + +// fastValuesAncestorKey is used to retrieve the closest fastValuesCtx ancestor. +type fastValuesAncestorKey struct{} + +// Value is part of the context.Context interface. +func (c *fastValuesCtx) Value(key any) any { + if _, ok := key.(fastValuesAncestorKey); ok { + return c + } + return c.Context.Value(key) +} + +func init() { + // Set up log tags to use fast values. + logtags.OverrideContextFuncs(LogTagsFromContext, WithLogTags) +} + +var LogTagsKey = RegisterFastValueKey() + +func LogTagsFromContext(ctx context.Context) *logtags.Buffer { + if v := FastValue(ctx, LogTagsKey); v != nil { + return v.(*logtags.Buffer) + } + return nil +} + +func WithLogTags(ctx context.Context, tags *logtags.Buffer) context.Context { + return WithFastValue(ctx, LogTagsKey, tags) +} + +// AddLogTags returns a context that has log the tags in the given context plus +// another set of log tags. Tags are deduplicated (see logtags.Buffer.AddTags). +func AddLogTags(ctx context.Context, tags *logtags.Buffer) context.Context { + b := LogTagsFromContext(ctx) + newB := b.Merge(tags) + if newB == b { + return ctx + } + return WithLogTags(ctx, newB) +} diff --git a/pkg/util/ctxutil/fast_value_test.go b/pkg/util/ctxutil/fast_value_test.go new file mode 100644 index 000000000000..c3f8c5f4cad6 --- /dev/null +++ b/pkg/util/ctxutil/fast_value_test.go @@ -0,0 +1,49 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package ctxutil + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +var k1 = RegisterFastValueKey() +var k2 = RegisterFastValueKey() +var k3 = RegisterFastValueKey() + +func TestFastValues(t *testing.T) { + ctx := context.Background() + + c1 := WithFastValue(ctx, k1, "val1") + require.Equal(t, "val1", FastValue(c1, k1)) + require.Nil(t, FastValue(c1, k2)) + + c2 := WithFastValue(c1, k2, "val2") + require.Equal(t, "val1", FastValue(c2, k1)) + require.Equal(t, "val2", FastValue(c2, k2)) + + // Verify that the values are propagated through other contexts. + c3, cancel := context.WithTimeout(c2, time.Hour) + defer cancel() + require.Equal(t, "val1", FastValue(c3, k1)) + require.Equal(t, "val2", FastValue(c3, k2)) + + b := WithFastValues(c3) + b.Set(k1, "val1-updated") + b.Set(k3, "val3") + c4 := b.Finish() + require.Equal(t, "val1-updated", FastValue(c4, k1)) + require.Equal(t, "val2", FastValue(c4, k2)) + require.Equal(t, "val3", FastValue(c4, k3)) + + c5 := WithFastValue(c4, k1, nil) + require.Nil(t, FastValue(c5, k1)) + require.Equal(t, "val2", FastValue(c5, k2)) + require.Equal(t, "val3", FastValue(c5, k3)) +} diff --git a/pkg/util/grpcutil/BUILD.bazel b/pkg/util/grpcutil/BUILD.bazel index de6230fd1125..fb9f16c6a439 100644 --- a/pkg/util/grpcutil/BUILD.bazel +++ b/pkg/util/grpcutil/BUILD.bazel @@ -13,6 +13,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb", + "//pkg/util/ctxutil", "//pkg/util/log", "//pkg/util/log/severity", "//pkg/util/netutil", diff --git a/pkg/util/grpcutil/grpc_util.go b/pkg/util/grpcutil/grpc_util.go index 800cc40d9425..f7c390169257 100644 --- a/pkg/util/grpcutil/grpc_util.go +++ b/pkg/util/grpcutil/grpc_util.go @@ -11,6 +11,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" "github.com/cockroachdb/cockroach/pkg/util/netutil" "github.com/cockroachdb/errors" "google.golang.org/grpc/codes" @@ -24,19 +25,19 @@ var ErrConnectionInterrupted = errors.New(errConnectionInterruptedMsg) const errConnectionInterruptedMsg = "connection interrupted (did the remote node shut down or are there networking issues?)" -type localRequestKey struct{} +var localRequestKey = ctxutil.RegisterFastValueKey() // NewLocalRequestContext returns a Context that can be used for local // (in-process) RPC requests performed by the InternalClientAdapter. The ctx // carries information about what tenant (if any) is the client of the RPC. The // auth interceptor uses this information to authorize the tenant. func NewLocalRequestContext(ctx context.Context, tenantID roachpb.TenantID) context.Context { - return context.WithValue(ctx, localRequestKey{}, tenantID) + return ctxutil.WithFastValue(ctx, localRequestKey, tenantID) } // IsLocalRequestContext returns true if this context is marked for local (in-process) use. func IsLocalRequestContext(ctx context.Context) (roachpb.TenantID, bool) { - val := ctx.Value(localRequestKey{}) + val := ctxutil.FastValue(ctx, localRequestKey) if val == nil { return roachpb.TenantID{}, false } diff --git a/pkg/util/log/BUILD.bazel b/pkg/util/log/BUILD.bazel index c45fc31c9ad5..e1b1f4b70823 100644 --- a/pkg/util/log/BUILD.bazel +++ b/pkg/util/log/BUILD.bazel @@ -63,6 +63,7 @@ go_library( "//pkg/util", "//pkg/util/allstacks", "//pkg/util/caller", + "//pkg/util/ctxutil", "//pkg/util/debugutil", "//pkg/util/encoding/encodingtype", "//pkg/util/envutil", diff --git a/pkg/util/log/ambient_context.go b/pkg/util/log/ambient_context.go index 01d216a875ed..a53d02086e8b 100644 --- a/pkg/util/log/ambient_context.go +++ b/pkg/util/log/ambient_context.go @@ -9,6 +9,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/base/serverident" + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/logtags" ) @@ -111,22 +112,34 @@ func (ac *AmbientContext) ResetAndAnnotateCtx(ctx context.Context) context.Conte } return ctx default: - ctx = logtags.WithTags(ctx, ac.tags) + bld := ctxutil.WithFastValues(ctx) + if ac.tags != nil { + bld.Set(ctxutil.LogTagsKey, ac.tags) + } if ac.ServerIDs != nil { - ctx = serverident.ContextWithServerIdentification(ctx, ac.ServerIDs) + bld.Set(serverident.ServerIdentificationContextKey, ac.ServerIDs) } - return ctx + return bld.Finish() } } func (ac *AmbientContext) annotateCtxInternal(ctx context.Context) context.Context { + bld := ctxutil.WithFastValues(ctx) if ac.tags != nil { - ctx = logtags.AddTags(ctx, ac.tags) + if v := bld.Get(ctxutil.LogTagsKey); v != nil { + existing := v.(*logtags.Buffer) + newTags := existing.Merge(ac.tags) + if newTags != existing { + bld.Set(ctxutil.LogTagsKey, newTags) + } + } else { + bld.Set(ctxutil.LogTagsKey, ac.tags) + } } - if ac.ServerIDs != nil && serverident.ServerIdentificationFromContext(ctx) == nil { - ctx = serverident.ContextWithServerIdentification(ctx, ac.ServerIDs) + if ac.ServerIDs != nil && bld.Get(serverident.ServerIdentificationContextKey) == nil { + bld.Set(serverident.ServerIdentificationContextKey, ac.ServerIDs) } - return ctx + return bld.Finish() } // AnnotateCtxWithSpan annotates the given context with the information in @@ -146,12 +159,7 @@ func (ac *AmbientContext) AnnotateCtxWithSpan( ctx = ac.backgroundCtx } default: - if ac.tags != nil { - ctx = logtags.AddTags(ctx, ac.tags) - } - if ac.ServerIDs != nil && serverident.ServerIdentificationFromContext(ctx) == nil { - ctx = serverident.ContextWithServerIdentification(ctx, ac.ServerIDs) - } + ctx = ac.annotateCtxInternal(ctx) } return tracing.ChildSpan(ctx, opName) diff --git a/pkg/util/tracing/BUILD.bazel b/pkg/util/tracing/BUILD.bazel index 2cbbc0023146..7230991e3ba8 100644 --- a/pkg/util/tracing/BUILD.bazel +++ b/pkg/util/tracing/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/settings", "//pkg/util/allstacks", "//pkg/util/buildutil", + "//pkg/util/ctxutil", "//pkg/util/debugutil", "//pkg/util/envutil", "//pkg/util/iterutil", diff --git a/pkg/util/tracing/context.go b/pkg/util/tracing/context.go index 1b31c7a7caea..14df00f557b4 100644 --- a/pkg/util/tracing/context.go +++ b/pkg/util/tracing/context.go @@ -5,9 +5,13 @@ package tracing -import "context" +import ( + "context" -type activeSpanKey struct{} + "github.com/cockroachdb/cockroach/pkg/util/ctxutil" +) + +var activeSpanKey = ctxutil.RegisterFastValueKey() // 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, @@ -17,9 +21,8 @@ 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 + if v := ctxutil.FastValue(ctx, activeSpanKey); v != nil { + return v.(*Span) } return nil } @@ -56,7 +59,7 @@ func maybeWrapCtx(ctx context.Context, sp *Span) (context.Context, *Span) { return ctx, sp } } - return context.WithValue(ctx, activeSpanKey{}, sp), sp + return ctxutil.WithFastValue(ctx, activeSpanKey, sp), sp } // ContextWithSpan returns a Context wrapping the supplied Span.