Skip to content

Commit

Permalink
ctxutil: introduce fast values
Browse files Browse the repository at this point in the history
This commit introduces a custom context type that can be used to more
efficiently associate values to contexts.

Instead of using arbitrary objects as keys, the keys are a small set
of integers. Any key must be globally registered first, and there is a
limited number that can be registered.

The `fastValuesCtx` stores *all* values for these keys, so traversing
the context hierarchy is not necessary. We also provide a way to add
multiple values at the same time, which saves on allocations.

I looked at before and after profiles in sysbench. Before:
 - CPU usage:
  context.value 0.6%
  context.WithValue 0.3%
  total 0.9%
 - Allocs:
  context.WithValue: 3.5%

After:
 - CPU usage:
  context.value + ctxutil.FastValue 0.5%
  context.WithValue 0.1%
  ctxutil.WithFastValue(s) 0.1%
  total 0.7%
 - Allocs:
  context.WithValue: 0.2%
  context.WithFastValue: 0.6%
  ctxutil.FastValuesBuilder.Finish: 1.4%
  total 2.2%

I will investigate improving on this, perhaps with providing a
pre-allocated context in codepaths where this is possible.

Informs: cockroachdb#136581
Release note: None
  • Loading branch information
RaduBerinde committed Dec 13, 2024
1 parent 7f44c49 commit 882ecf9
Show file tree
Hide file tree
Showing 18 changed files with 305 additions and 53 deletions.
1 change: 1 addition & 0 deletions pkg/base/serverident/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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"],
)
20 changes: 13 additions & 7 deletions pkg/base/serverident/server_ident.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,33 +5,39 @@

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
// the ctx later.
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)
}

Expand All @@ -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
)

Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/sqlproxyccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
11 changes: 6 additions & 5 deletions pkg/ccl/sqlproxyccl/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/kvflowcontrol/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
Expand Down
7 changes: 4 additions & 3 deletions pkg/kv/kvserver/kvflowcontrol/kvflowcontrol.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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.
Expand All @@ -445,15 +446,15 @@ 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
}

// 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
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
17 changes: 7 additions & 10 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -4714,38 +4715,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)
}
Expand Down
8 changes: 6 additions & 2 deletions pkg/util/ctxutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
4 changes: 2 additions & 2 deletions pkg/util/ctxutil/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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)
Expand Down
Loading

0 comments on commit 882ecf9

Please sign in to comment.