diff --git a/pkg/ccl/kvccl/kvtenantccl/proxy.go b/pkg/ccl/kvccl/kvtenantccl/proxy.go index 27054a998b11..d8dc8ded511f 100644 --- a/pkg/ccl/kvccl/kvtenantccl/proxy.go +++ b/pkg/ccl/kvccl/kvtenantccl/proxy.go @@ -31,6 +31,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/syncutil/singleflight" "github.com/cockroachdb/errors" "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" ) func init() { @@ -295,13 +297,17 @@ func (p *Proxy) RangeLookup( PrefetchReverse: useReverseScan, }) if err != nil { - // Soft RPC error. Drop client and retry. log.Warningf(ctx, "error issuing RangeLookup RPC: %v", err) + if status.Code(err) == codes.Unauthenticated { + // Authentication error. Propagate. + return nil, nil, err + } + // Soft RPC error. Drop client and retry. p.tryForgetClient(ctx, client) continue } if resp.Error != nil { - // Hard logical error. + // Hard logical error. Propagate. return nil, nil, resp.Error.GoError() } return resp.Descriptors, resp.PrefetchedDescriptors, nil diff --git a/pkg/roachpb/tenant.go b/pkg/roachpb/tenant.go index f3c48f849758..8bf79e652b02 100644 --- a/pkg/roachpb/tenant.go +++ b/pkg/roachpb/tenant.go @@ -11,6 +11,7 @@ package roachpb import ( + "context" "math" "strconv" ) @@ -76,3 +77,16 @@ func checkValid(id uint64) { func IsSystemTenantID(id uint64) bool { return id == SystemTenantID.ToUint64() } + +type tenantKey struct{} + +// NewContextForTenant creates a new context with tenant information attached. +func NewContextForTenant(ctx context.Context, tenID TenantID) context.Context { + return context.WithValue(ctx, tenantKey{}, tenID) +} + +// TenantFromContext returns the tenant information in ctx if it exists. +func TenantFromContext(ctx context.Context) (tenID TenantID, ok bool) { + tenID, ok = ctx.Value(tenantKey{}).(TenantID) + return +} diff --git a/pkg/rpc/auth.go b/pkg/rpc/auth.go new file mode 100644 index 000000000000..221ae041b330 --- /dev/null +++ b/pkg/rpc/auth.go @@ -0,0 +1,101 @@ +// 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 rpc + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/peer" + "google.golang.org/grpc/status" +) + +var errTLSInfoMissing = authError("TLSInfo is not available in request context") + +func authError(msg string) error { + return status.Error(codes.Unauthenticated, msg) +} + +func authErrorf(format string, a ...interface{}) error { + return status.Errorf(codes.Unauthenticated, format, a...) +} + +// auth is a policy that performs authentication and authorization for unary and +// streaming RPC invocations. An auth enforces its policy through a pair of gRPC +// interceptors that it exports. +type auth interface { + // AuthUnary returns an interceptor to validate unary RPCs. + AuthUnary() grpc.UnaryServerInterceptor + + // AuthUnary returns an interceptor to validate streaming RPCs. + AuthStream() grpc.StreamServerInterceptor +} + +// kvAuth is the standard auth policy used for RPCs sent to an RPC server. It +// validates that client TLS certificate provided by the incoming connection +// contains a sufficiently privileged user. +type kvAuth struct{} + +// kvAuth implements the auth interface. +func (a kvAuth) AuthUnary() grpc.UnaryServerInterceptor { return a.unaryInterceptor } +func (a kvAuth) AuthStream() grpc.StreamServerInterceptor { return a.streamInterceptor } + +func (a kvAuth) unaryInterceptor( + ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler, +) (interface{}, error) { + if err := a.requireSuperUser(ctx); err != nil { + return nil, err + } + return handler(ctx, req) +} + +func (a kvAuth) streamInterceptor( + srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler, +) error { + if err := a.requireSuperUser(ss.Context()); err != nil { + return err + } + return handler(srv, ss) +} + +func (a kvAuth) requireSuperUser(ctx context.Context) error { + // TODO(marc): grpc's authentication model (which gives credential access in + // the request handler) doesn't really fit with the current design of the + // security package (which assumes that TLS state is only given at connection + // time) - that should be fixed. + if grpcutil.IsLocalRequestContext(ctx) { + // This is an in-process request. Bypass authentication check. + } else if peer, ok := peer.FromContext(ctx); ok { + if tlsInfo, ok := peer.AuthInfo.(credentials.TLSInfo); ok { + certUsers, err := security.GetCertificateUsers(&tlsInfo.State) + if err != nil { + return err + } + // TODO(benesch): the vast majority of RPCs should be limited to just + // NodeUser. This is not a security concern, as RootUser has access to + // read and write all data, merely good hygiene. For example, there is + // no reason to permit the root user to send raw Raft RPCs. + if !security.ContainsUser(security.NodeUser, certUsers) && + !security.ContainsUser(security.RootUser, certUsers) { + return authErrorf("user %s is not allowed to perform this RPC", certUsers) + } + } else { + // TODO DURING REVIEW: is this a typo or intentional? + } + } else { + return errTLSInfoMissing + } + return nil +} diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go new file mode 100644 index 000000000000..3443a5504529 --- /dev/null +++ b/pkg/rpc/auth_tenant.go @@ -0,0 +1,245 @@ +// 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 rpc + +import ( + "context" + "strconv" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/logtags" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/peer" +) + +// tenantAuth is an auth policy used for RPCs sent to a node's tenant RPC +// server. It validates that client TLS certificates provide a tenant ID and +// that the RPC being invoked is compatable with that tenant. +type tenantAuth struct{} + +// kvAuth implements the auth interface. +func (a tenantAuth) AuthUnary() grpc.UnaryServerInterceptor { return a.unaryInterceptor } +func (a tenantAuth) AuthStream() grpc.StreamServerInterceptor { return a.streamInterceptor } + +func (a tenantAuth) unaryInterceptor( + ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler, +) (interface{}, error) { + tenID, err := a.tenantFromCert(ctx) + if err != nil { + return nil, err + } + ctx = contextWithTenant(ctx, tenID) + if err := a.authRequest(ctx, tenID, info.FullMethod, req); err != nil { + return nil, err + } + return handler(ctx, req) +} + +func (a tenantAuth) streamInterceptor( + srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler, +) error { + ctx := ss.Context() + tenID, err := a.tenantFromCert(ctx) + if err != nil { + return err + } + ctx = contextWithTenant(ctx, tenID) + return handler(srv, &wrappedServerStream{ + ServerStream: ss, + ctx: ctx, + recv: func(m interface{}) error { + if err := ss.RecvMsg(m); err != nil { + return err + } + // 'm' is now populated and contains the request from the client. + return a.authRequest(ctx, tenID, info.FullMethod, m) + }, + }) +} + +func (a tenantAuth) tenantFromCert(ctx context.Context) (roachpb.TenantID, error) { + peer, ok := peer.FromContext(ctx) + if !ok { + return roachpb.TenantID{}, errTLSInfoMissing + } + + tlsInfo, ok := peer.AuthInfo.(credentials.TLSInfo) + if !ok { + return roachpb.TenantID{}, errTLSInfoMissing + } + + commonName := tlsInfo.State.PeerCertificates[0].Subject.CommonName + tenID, err := strconv.ParseUint(commonName, 10, 64) + if err != nil { + return roachpb.TenantID{}, authErrorf("could not parse tenant ID from Common Name (CN): %s", err) + } + if tenID < roachpb.MinTenantID.ToUint64() || tenID > roachpb.MaxTenantID.ToUint64() { + return roachpb.TenantID{}, authErrorf("invalid tenant ID %d in Common Name (CN)", tenID) + } + return roachpb.MakeTenantID(tenID), nil +} + +func (a tenantAuth) authRequest( + ctx context.Context, tenID roachpb.TenantID, fullMethod string, req interface{}, +) error { + switch fullMethod { + case "/cockroach.roachpb.Internal/Batch": + return a.authBatch(tenID, req.(*roachpb.BatchRequest)) + + case "/cockroach.roachpb.Internal/RangeLookup": + return a.authRangeLookup(tenID, req.(*roachpb.RangeLookupRequest)) + + case "/cockroach.roachpb.Internal/RangeFeed": + return a.authRangeFeed(tenID, req.(*roachpb.RangeFeedRequest)) + + case "/cockroach.roachpb.Internal/GossipSubscription": + return a.authGossipSubscription(tenID, req.(*roachpb.GossipSubscriptionRequest)) + + case "/cockroach.rpc.Heartbeat/Ping": + return nil // no authorization + + default: + return authErrorf("unknown method %q", fullMethod) + } +} + +// authBatch authorizes the provided tenant to invoke the Batch RPC with the +// provided args. +func (a tenantAuth) authBatch(tenID roachpb.TenantID, args *roachpb.BatchRequest) error { + rSpan, err := keys.Range(args.Requests) + if err != nil { + return authError(err.Error()) + } + tenSpan := tenantPrefix(tenID) + if tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) { + return nil + } + for _, allow := range batchSpanAllowlist { + if rSpan.Equal(allow) { + return nil + } + } + return authErrorf("requested key span %s not fully contained in tenant keypace %s", rSpan, tenSpan) +} + +// batchSpanAllowlist contains spans outside of a tenant's keyspace that Batch +// RPC invocations are allowed to touch. +var batchSpanAllowlist = []roachpb.RSpan{ + // TODO(nvanbenschoten): Explore whether we can get rid of this by no longer + // reading this key in sqlServer.start. + { + Key: roachpb.RKey(keys.BootstrapVersionKey), + EndKey: roachpb.RKey(keys.BootstrapVersionKey.Next()), + }, +} + +// authRangeLookup authorizes the provided tenant to invoke the RangeLookup RPC +// with the provided args. +func (a tenantAuth) authRangeLookup( + tenID roachpb.TenantID, args *roachpb.RangeLookupRequest, +) error { + tenSpan := tenantPrefix(tenID) + if tenSpan.ContainsKey(args.Key) { + return nil + } + for _, allow := range rangeLookupKeyAllowlist { + if args.Key.Equal(allow) { + return nil + } + } + return authErrorf("requested key %s not fully contained in tenant keypace %s", args.Key, tenSpan) +} + +// rangeLookupKeyAllowlist contains keys outside of a tenant's keyspace that +// RangeLookup RPC invocations are allowed to touch. +var rangeLookupKeyAllowlist = []roachpb.Key{ + // TODO(nvanbenschoten): Explore whether we can get rid of this by no longer + // reading this key in sqlServer.start. + keys.BootstrapVersionKey, +} + +// authRangeFeed authorizes the provided tenant to invoke the RangeFeed RPC with +// the provided args. +func (a tenantAuth) authRangeFeed(tenID roachpb.TenantID, args *roachpb.RangeFeedRequest) error { + rSpan, err := keys.SpanAddr(args.Span) + if err != nil { + return authError(err.Error()) + } + tenSpan := tenantPrefix(tenID) + if !tenSpan.ContainsKeyRange(rSpan.Key, rSpan.EndKey) { + return authErrorf("requested key span %s not fully contained in tenant keypace %s", rSpan, tenSpan) + } + return nil +} + +// authGossipSubscription authorizes the provided tenant to invoke the +// GossipSubscription RPC with the provided args. +func (a tenantAuth) authGossipSubscription( + tenID roachpb.TenantID, args *roachpb.GossipSubscriptionRequest, +) error { + for _, pat := range args.Patterns { + allowed := false + for _, allow := range gossipSubscriptionPatternAllowlist { + if pat == allow { + allowed = true + break + } + } + if !allowed { + return authErrorf("requested pattern %q not permitted", pat) + } + } + return nil +} + +// gossipSubscriptionPatternAllowlist contains keys outside of a tenant's +// keyspace that GossipSubscription RPC invocations are allowed to touch. +// WIP: can't import gossip directly. +var gossipSubscriptionPatternAllowlist = []string{ + "node:.*", + "system-db", +} + +func contextWithTenant(ctx context.Context, tenID roachpb.TenantID) context.Context { + ctx = roachpb.NewContextForTenant(ctx, tenID) + ctx = logtags.AddTag(ctx, "tenant", tenID.String()) + return ctx +} + +func tenantPrefix(tenID roachpb.TenantID) roachpb.RSpan { + // TODO(nvanbenschoten): consider caching this span. + prefix := roachpb.RKey(keys.MakeTenantPrefix(tenID)) + return roachpb.RSpan{ + Key: prefix, + EndKey: prefix.PrefixEnd(), + } +} + +// wrappedServerStream is a thin wrapper around grpc.ServerStream that allows +// modifying its context and overriding its RecvMsg method. It can be used to +// intercept each messsage and inject custom validation logic. +type wrappedServerStream struct { + grpc.ServerStream + ctx context.Context + recv func(interface{}) error +} + +// Context overrides the nested grpc.ServerStream.Context(). +func (ss *wrappedServerStream) Context() context.Context { + return ss.ctx +} + +// RecvMsg overrides the nested grpc.ServerStream.RecvMsg(). +func (ss *wrappedServerStream) RecvMsg(m interface{}) error { + return ss.recv(m) +} diff --git a/pkg/rpc/auth_test.go b/pkg/rpc/auth_test.go new file mode 100644 index 000000000000..362456b9b50c --- /dev/null +++ b/pkg/rpc/auth_test.go @@ -0,0 +1,326 @@ +// 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 rpc + +import ( + "context" + "crypto/tls" + "crypto/x509" + "crypto/x509/pkix" + "io" + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/metadata" + "google.golang.org/grpc/peer" + "google.golang.org/grpc/status" +) + +// mockServerStream is an implementation of grpc.ServerStream that receives a +// list of integers. +type mockServerStream []int + +func (mockServerStream) SetHeader(metadata.MD) error { panic("unimplemented") } +func (mockServerStream) SendHeader(metadata.MD) error { panic("unimplemented") } +func (mockServerStream) SetTrailer(metadata.MD) { panic("unimplemented") } +func (mockServerStream) Context() context.Context { panic("unimplemented") } +func (mockServerStream) SendMsg(m interface{}) error { panic("unimplemented") } +func (s *mockServerStream) RecvMsg(m interface{}) error { + if len(*s) == 0 { + return io.EOF + } + *(m.(*int)) = (*s)[0] + *s = (*s)[1:] + return nil +} + +func TestWrappedServerStream(t *testing.T) { + defer leaktest.AfterTest(t)() + ss := mockServerStream{1, 2, 3} + ctx := context.WithValue(context.Background(), struct{}{}, "v") + + var recv int + wrapped := &wrappedServerStream{ + ServerStream: &ss, + ctx: ctx, + recv: func(m interface{}) error { + if err := ss.RecvMsg(m); err != nil { + return err + } + recv = *(m.(*int)) + return nil + }, + } + + // Context() returns the wrapped context. + require.Equal(t, ctx, wrapped.Context()) + + // RecvMsg calls the instrumented function. + var i int + require.NoError(t, wrapped.RecvMsg(&i)) + require.Equal(t, 1, i) + require.Equal(t, 1, recv) + + // The wrapped stream can be used directly. + require.NoError(t, wrapped.ServerStream.RecvMsg(&i)) + require.Equal(t, 2, i) + require.Equal(t, 1, recv) + + require.NoError(t, wrapped.RecvMsg(&i)) + require.Equal(t, 3, i) + require.Equal(t, 3, recv) + + // io.EOF propagated correctly. Message not updated. + require.Equal(t, io.EOF, wrapped.RecvMsg(&i)) + require.Equal(t, 3, i) + require.Equal(t, 3, recv) +} + +func TestTenantFromCert(t *testing.T) { + defer leaktest.AfterTest(t)() + for _, tc := range []struct { + commonName string + expTenID roachpb.TenantID + expErr string + }{ + {commonName: "10", expTenID: roachpb.MakeTenantID(10)}, + {commonName: roachpb.MinTenantID.String(), expTenID: roachpb.MinTenantID}, + {commonName: roachpb.MaxTenantID.String(), expTenID: roachpb.MaxTenantID}, + {commonName: roachpb.SystemTenantID.String() /* "system" */, expErr: `could not parse tenant ID from Common Name \(CN\)`}, + {commonName: "-1", expErr: `could not parse tenant ID from Common Name \(CN\)`}, + {commonName: "0", expErr: `invalid tenant ID 0 in Common Name \(CN\)`}, + {commonName: "1", expErr: `invalid tenant ID 1 in Common Name \(CN\)`}, + {commonName: "root", expErr: `could not parse tenant ID from Common Name \(CN\)`}, + {commonName: "other", expErr: `could not parse tenant ID from Common Name \(CN\)`}, + } { + t.Run(tc.commonName, func(t *testing.T) { + cert := &x509.Certificate{ + Subject: pkix.Name{CommonName: tc.commonName}, + } + tlsInfo := credentials.TLSInfo{ + State: tls.ConnectionState{ + PeerCertificates: []*x509.Certificate{cert}, + }, + } + p := peer.Peer{AuthInfo: tlsInfo} + ctx := peer.NewContext(context.Background(), &p) + + tenID, err := tenantAuth{}.tenantFromCert(ctx) + if tc.expErr == "" { + require.Equal(t, tc.expTenID, tenID) + require.NoError(t, err) + } else { + require.Zero(t, tenID) + require.Error(t, err) + require.Equal(t, codes.Unauthenticated, status.Code(err)) + require.Regexp(t, tc.expErr, err) + } + }) + } +} + +func TestTenantAuthRequest(t *testing.T) { + defer leaktest.AfterTest(t)() + tenID := roachpb.MakeTenantID(10) + prefix := func(tenID uint64, key string) string { + tenPrefix := keys.MakeTenantPrefix(roachpb.MakeTenantID(tenID)) + return string(append(tenPrefix, []byte(key)...)) + } + makeSpan := func(key string, endKey ...string) roachpb.Span { + s := roachpb.Span{Key: roachpb.Key(key)} + if len(endKey) > 1 { + t.Fatalf("unexpected endKey vararg %v", endKey) + } else if len(endKey) == 1 { + s.EndKey = roachpb.Key(endKey[0]) + } + return s + } + makeReq := func(key string, endKey ...string) roachpb.Request { + s := makeSpan(key, endKey...) + h := roachpb.RequestHeaderFromSpan(s) + return &roachpb.ScanRequest{RequestHeader: h} + } + makeReqs := func(reqs ...roachpb.Request) []roachpb.RequestUnion { + ru := make([]roachpb.RequestUnion, len(reqs)) + for i, r := range reqs { + ru[i].MustSetInner(r) + } + return ru + } + + const noError = "" + for method, tests := range map[string][]struct { + req interface{} + expErr string + }{ + "/cockroach.roachpb.Internal/Batch": { + { + req: &roachpb.BatchRequest{}, + expErr: `requested key span /Max not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq("a", "b"), + )}, + expErr: `requested key span {a-b} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq(prefix(5, "a"), prefix(5, "b")), + )}, + expErr: `requested key span /Tenant/5"{a"-b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq(prefix(10, "a"), prefix(10, "b")), + )}, + expErr: noError, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq(prefix(50, "a"), prefix(50, "b")), + )}, + expErr: `requested key span /Tenant/50"{a"-b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq("a", "b"), + makeReq(prefix(5, "a"), prefix(5, "b")), + )}, + expErr: `requested key span {a-/Tenant/5"b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq(prefix(5, "a"), prefix(5, "b")), + makeReq(prefix(10, "a"), prefix(10, "b")), + )}, + expErr: `requested key span /Tenant/{5"a"-10"b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq("a", prefix(10, "b")), + )}, + expErr: `requested key span {a-/Tenant/10"b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.BatchRequest{Requests: makeReqs( + makeReq(prefix(10, "a"), prefix(20, "b")), + )}, + expErr: `requested key span /Tenant/{10"a"-20"b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + }, + "/cockroach.roachpb.Internal/RangeLookup": { + { + req: &roachpb.RangeLookupRequest{}, + expErr: `requested key /Min not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeLookupRequest{Key: roachpb.RKey("a")}, + expErr: `requested key "a" not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeLookupRequest{Key: roachpb.RKey(prefix(5, "a"))}, + expErr: `requested key /Tenant/5"a" not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeLookupRequest{Key: roachpb.RKey(prefix(10, "a"))}, + expErr: noError, + }, + { + req: &roachpb.RangeLookupRequest{Key: roachpb.RKey(prefix(50, "a"))}, + expErr: `requested key /Tenant/50"a" not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + }, + "/cockroach.roachpb.Internal/RangeFeed": { + { + req: &roachpb.RangeFeedRequest{}, + expErr: `requested key span /Min not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeFeedRequest{Span: makeSpan("a", "b")}, + expErr: `requested key span {a-b} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(5, "a"), prefix(5, "b"))}, + expErr: `requested key span /Tenant/5"{a"-b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(10, "a"), prefix(10, "b"))}, + expErr: noError, + }, + { + req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(50, "a"), prefix(50, "b"))}, + expErr: `requested key span /Tenant/50"{a"-b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeFeedRequest{Span: makeSpan("a", prefix(10, "b"))}, + expErr: `requested key span {a-/Tenant/10"b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + { + req: &roachpb.RangeFeedRequest{Span: makeSpan(prefix(10, "a"), prefix(20, "b"))}, + expErr: `requested key span /Tenant/{10"a"-20"b"} not fully contained in tenant keypace /Tenant/1{0-1}`, + }, + }, + "/cockroach.roachpb.Internal/GossipSubscription": { + { + req: &roachpb.GossipSubscriptionRequest{}, + expErr: noError, + }, + { + req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"node:.*"}}, + expErr: noError, + }, + { + req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"system-db"}}, + expErr: noError, + }, + { + req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"table-stat-added"}}, + expErr: `requested pattern "table-stat-added" not permitted`, + }, + { + req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"node:.*", "system-db"}}, + expErr: noError, + }, + { + req: &roachpb.GossipSubscriptionRequest{Patterns: []string{"node:.*", "system-db", "table-stat-added"}}, + expErr: `requested pattern "table-stat-added" not permitted`, + }, + }, + "/cockroach.rpc.Heartbeat/Ping": { + {req: &PingRequest{}, expErr: noError}, + }, + "/cockroach.rpc.Testing/Foo": { + {req: "req", expErr: `unknown method "/cockroach.rpc.Testing/Foo"`}, + }, + } { + t.Run(method, func(t *testing.T) { + for _, tc := range tests { + t.Run("", func(t *testing.T) { + ctx := context.Background() + err := tenantAuth{}.authRequest(ctx, tenID, method, tc.req) + if tc.expErr == noError { + require.NoError(t, err) + } else { + require.Error(t, err) + require.Equal(t, codes.Unauthenticated, status.Code(err)) + require.Regexp(t, tc.expErr, err) + } + }) + } + }) + } +} diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 92476bc54bd1..9ff2addfe9c0 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -26,7 +26,6 @@ import ( circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/envutil" @@ -49,7 +48,6 @@ import ( "google.golang.org/grpc/encoding" encodingproto "google.golang.org/grpc/encoding/proto" "google.golang.org/grpc/metadata" - "google.golang.org/grpc/peer" ) func init() { @@ -114,34 +112,6 @@ func spanInclusionFuncForClient( return parentSpanCtx != nil && !tracing.IsNoopContext(parentSpanCtx) } -func requireSuperUser(ctx context.Context) error { - // TODO(marc): grpc's authentication model (which gives credential access in - // the request handler) doesn't really fit with the current design of the - // security package (which assumes that TLS state is only given at connection - // time) - that should be fixed. - if grpcutil.IsLocalRequestContext(ctx) { - // This is an in-process request. Bypass authentication check. - } else if peer, ok := peer.FromContext(ctx); ok { - if tlsInfo, ok := peer.AuthInfo.(credentials.TLSInfo); ok { - certUsers, err := security.GetCertificateUsers(&tlsInfo.State) - if err != nil { - return err - } - // TODO(benesch): the vast majority of RPCs should be limited to just - // NodeUser. This is not a security concern, as RootUser has access to - // read and write all data, merely good hygiene. For example, there is - // no reason to permit the root user to send raw Raft RPCs. - if !security.ContainsUser(security.NodeUser, certUsers) && - !security.ContainsUser(security.RootUser, certUsers) { - return errors.Errorf("user %s is not allowed to perform this RPC", certUsers) - } - } - } else { - return errors.New("internal authentication error: TLSInfo is not available in request context") - } - return nil -} - type serverOpts struct { interceptor func(fullMethod string) error tenant bool @@ -158,8 +128,6 @@ func ForTenant(opts *serverOpts) { opts.tenant = true } -var _ ServerOption = ForTenant - // WithInterceptor adds an additional interceptor. The interceptor is called before // streaming and unary RPCs and may inject an error. // @@ -224,35 +192,14 @@ func NewServer(ctx *Context, opts ...ServerOption) *grpc.Server { var streamInterceptor []grpc.StreamServerInterceptor if !ctx.Config.Insecure { - var authHook func(ctx context.Context) error + var a auth if !o.tenant { - authHook = requireSuperUser + a = kvAuth{} } else { - authHook = func(ctx context.Context) error { - // TODO(tbg): pull the tenant ID from the incoming - // certificate and validate that the request is - // admissible. - return nil - } + a = tenantAuth{} } - - // TODO(tbg): do something else if `o.tenant`. - unaryInterceptor = append(unaryInterceptor, func( - ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler, - ) (interface{}, error) { - if err := authHook(ctx); err != nil { - return nil, err - } - return handler(ctx, req) - }) - streamInterceptor = append(streamInterceptor, func( - srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler, - ) error { - if err := authHook(stream.Context()); err != nil { - return err - } - return handler(srv, stream) - }) + unaryInterceptor = append(unaryInterceptor, a.AuthUnary()) + streamInterceptor = append(streamInterceptor, a.AuthStream()) } if o.interceptor != nil {