From 4384a9251b8363005c802526dc110e3b6e9fd3b4 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Wed, 1 Feb 2023 00:22:50 +0100 Subject: [PATCH] grpcutil: new+fast FromIncomingContext variant If we want to properly leverage gRPC metadata, we need to find a way around the fact that the default `metatada.FromIncomingContext` is slow and expensive. This patch fixes it. ``` BenchmarkFromIncomingContext/stdlib-32 10987958 327.5 ns/op 432 B/op 3 allocs/op BenchmarkFromIncomingContext/fast-32 698772889 5.152 ns/op 0 B/op 0 allocs/op ``` Release note: None --- pkg/blobs/BUILD.bazel | 1 + pkg/blobs/service.go | 13 +-- pkg/server/authentication.go | 3 +- pkg/server/status.go | 4 +- pkg/util/grpcutil/BUILD.bazel | 4 + pkg/util/grpcutil/fast_metadata.go | 104 ++++++++++++++++++ pkg/util/grpcutil/fast_metadata_test.go | 35 ++++++ .../grpcinterceptor/grpc_interceptor.go | 2 +- 8 files changed, 155 insertions(+), 11 deletions(-) create mode 100644 pkg/util/grpcutil/fast_metadata.go create mode 100644 pkg/util/grpcutil/fast_metadata_test.go diff --git a/pkg/blobs/BUILD.bazel b/pkg/blobs/BUILD.bazel index 5f92aa6d63fb..1bee54a3b413 100644 --- a/pkg/blobs/BUILD.bazel +++ b/pkg/blobs/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/rpc", "//pkg/rpc/nodedialer", "//pkg/util/fileutil", + "//pkg/util/grpcutil", "//pkg/util/ioctx", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_errors//oserror", diff --git a/pkg/blobs/service.go b/pkg/blobs/service.go index 6d81076af68b..55a131753fa6 100644 --- a/pkg/blobs/service.go +++ b/pkg/blobs/service.go @@ -30,11 +30,11 @@ import ( "io" "github.com/cockroachdb/cockroach/pkg/blobs/blobspb" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/ioctx" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" "google.golang.org/grpc/codes" - "google.golang.org/grpc/metadata" "google.golang.org/grpc/status" ) @@ -63,20 +63,19 @@ func (s *Service) GetStream(req *blobspb.GetRequest, stream blobspb.Blob_GetStre // PutStream implements the gRPC service. func (s *Service) PutStream(stream blobspb.Blob_PutStreamServer) error { - md, ok := metadata.FromIncomingContext(stream.Context()) + filename, ok := grpcutil.FastFirstValueFromIncomingContext(stream.Context(), "filename") if !ok { - return errors.New("could not fetch metadata") + return errors.New("could not fetch metadata or no filename in metadata") } - filename := md.Get("filename") - if len(filename) < 1 || filename[0] == "" { - return errors.New("no filename in metadata") + if filename == "" { + return errors.New("invalid filename in metadata") } reader := newPutStreamReader(stream) defer reader.Close(stream.Context()) ctx, cancel := context.WithCancel(stream.Context()) defer cancel() - w, err := s.localStorage.Writer(ctx, filename[0]) + w, err := s.localStorage.Writer(ctx, filename) if err != nil { cancel() return err diff --git a/pkg/server/authentication.go b/pkg/server/authentication.go index f3ef7a5423bc..ac15f7d70552 100644 --- a/pkg/server/authentication.go +++ b/pkg/server/authentication.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/ui" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -307,7 +308,7 @@ func (s *authenticationServer) createSessionFor( func (s *authenticationServer) UserLogout( ctx context.Context, req *serverpb.UserLogoutRequest, ) (*serverpb.UserLogoutResponse, error) { - md, ok := metadata.FromIncomingContext(ctx) + md, ok := grpcutil.FastFromIncomingContext(ctx) if !ok { return nil, apiInternalError(ctx, fmt.Errorf("couldn't get incoming context")) } diff --git a/pkg/server/status.go b/pkg/server/status.go index 41ae7ee0a299..758e61a767c6 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -134,7 +134,7 @@ type metricMarshaler interface { } func propagateGatewayMetadata(ctx context.Context) context.Context { - if md, ok := metadata.FromIncomingContext(ctx); ok { + if md, ok := grpcutil.FastFromIncomingContext(ctx); ok { return metadata.NewOutgoingContext(ctx, md) } return ctx @@ -3549,7 +3549,7 @@ func marshalJSONResponse(value interface{}) (*serverpb.JSONResponse, error) { } func userFromContext(ctx context.Context) (res username.SQLUsername, err error) { - md, ok := metadata.FromIncomingContext(ctx) + md, ok := grpcutil.FastFromIncomingContext(ctx) if !ok { return username.RootUserName(), nil } diff --git a/pkg/util/grpcutil/BUILD.bazel b/pkg/util/grpcutil/BUILD.bazel index a57f60032df8..c2e9c3b35c87 100644 --- a/pkg/util/grpcutil/BUILD.bazel +++ b/pkg/util/grpcutil/BUILD.bazel @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "grpcutil", srcs = [ + "fast_metadata.go", "grpc_err_redaction.go", "grpc_log.go", "grpc_util.go", @@ -24,6 +25,7 @@ go_library( "@com_github_gogo_status//:status", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//grpclog", + "@org_golang_google_grpc//metadata", "@org_golang_google_grpc//status", ], ) @@ -32,6 +34,7 @@ go_test( name = "grpcutil_test", size = "small", srcs = [ + "fast_metadata_test.go", "grpc_err_redaction_test.go", "grpc_log_test.go", "grpc_util_test.go", @@ -53,6 +56,7 @@ go_test( "@org_golang_google_grpc//:go_default_library", "@org_golang_google_grpc//codes", "@org_golang_google_grpc//health/grpc_health_v1", + "@org_golang_google_grpc//metadata", ], ) diff --git a/pkg/util/grpcutil/fast_metadata.go b/pkg/util/grpcutil/fast_metadata.go new file mode 100644 index 000000000000..0b85281aa95f --- /dev/null +++ b/pkg/util/grpcutil/fast_metadata.go @@ -0,0 +1,104 @@ +// Copyright 2023 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 grpcutil + +import ( + "context" + "strings" + "time" + + "google.golang.org/grpc/metadata" +) + +// FastFromIncomingContext is a specialization of +// metadata.FromIncomingContext() which extracts the metadata.MD from +// the context, if any, by reference. Main differences: +// +// - This variant does not guarantee that all the MD keys are +// lowercase. This happens to be true when the MD is populated by +// gRPC itself on an incoming RPC call, but it may not be true for +// MD populated elsewhere. +// - The caller promises to not modify the returned MD -- the gRPC +// APIs assume that the map in the context remains constant. +func FastFromIncomingContext(ctx context.Context) (metadata.MD, bool) { + md, ok := ctx.Value(grpcIncomingKeyObj).(metadata.MD) + return md, ok +} + +// FastFirstValueFromIncomingContext is a specialization of +// metadata.ValueFromIncomingContext() which extracts the first string +// from the given metadata key, if it exists. No extra objects are +// allocated. The key is assumed to contain only ASCII characters. +func FastFirstValueFromIncomingContext(ctx context.Context, key string) (string, bool) { + md, ok := ctx.Value(grpcIncomingKeyObj).(metadata.MD) + if !ok { + return "", false + } + if v, ok := md[key]; ok { + if len(v) > 0 { + return v[0], true + } + return "", false + } + for k, v := range md { + // The letter casing may not have been set properly when MD was + // attached to the context. So we need to normalize it here. + // + // We add len(k) == len(key) to avoid the overhead of + // strings.ToLower when the keys of different length, because then + // they are guaranteed to not match anyway. This is the + // optimization that requires the key to be all ASCII, as + // generally ToLower() on non-ascii unicode can change the length + // of the string. + if len(k) == len(key) && strings.ToLower(k) == key { + if len(v) > 0 { + return v[0], true + } + return "", false + } + } + return "", false +} + +// grpcIncomingKeyObj is a copy of a value with the Go type +// `metadata.incomingKey{}` (from the grpc metadata package). We +// cannot construct an object of that type directly, but we can +// "steal" it by forcing the metadata package to give it to us: +// `metadata.FromIncomingContext` gives an instance of this object as +// parameter to the `Value` method of the context you give it as +// argument. We use a custom implementation of that to "steal" the +// argument of type `incomingKey{}` given to us that way. +var grpcIncomingKeyObj = func() interface{} { + var f fakeContext + _, _ = metadata.FromIncomingContext(&f) + if f.recordedKey == nil { + panic("ValueFromIncomingContext did not request a key") + } + return f.recordedKey +}() + +type fakeContext struct { + recordedKey interface{} +} + +var _ context.Context = (*fakeContext)(nil) + +// Value implements the context.Context interface and is our helper +// that "steals" an instance of the private type `incomingKey` in the +// grpc metadata package. +func (f *fakeContext) Value(keyObj interface{}) interface{} { + f.recordedKey = keyObj + return nil +} + +func (*fakeContext) Deadline() (time.Time, bool) { panic("unused") } +func (*fakeContext) Done() <-chan struct{} { panic("unused") } +func (*fakeContext) Err() error { panic("unused") } diff --git a/pkg/util/grpcutil/fast_metadata_test.go b/pkg/util/grpcutil/fast_metadata_test.go new file mode 100644 index 000000000000..a10232583e7d --- /dev/null +++ b/pkg/util/grpcutil/fast_metadata_test.go @@ -0,0 +1,35 @@ +// Copyright 2023 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 grpcutil + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/metadata" +) + +func TestFastFromIncomingContext(t *testing.T) { + defer leaktest.AfterTest(t)() + + md := metadata.MD{"hello": []string{"world", "universe"}} + + ctx := metadata.NewIncomingContext(context.Background(), md) + md2, ok := FastFromIncomingContext(ctx) + require.True(t, ok) + require.Equal(t, md2, md) + + v, ok := FastFirstValueFromIncomingContext(ctx, "hello") + require.True(t, ok) + require.Equal(t, v, "world") +} diff --git a/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go b/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go index 10c0b4d16b97..94fc1b9085b4 100644 --- a/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go +++ b/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go @@ -29,7 +29,7 @@ import ( func ExtractSpanMetaFromGRPCCtx( ctx context.Context, tracer *tracing.Tracer, ) (tracing.SpanMeta, error) { - md, ok := metadata.FromIncomingContext(ctx) + md, ok := grpcutil.FastFromIncomingContext(ctx) if !ok { return tracing.SpanMeta{}, nil }