From b4f4d6bf248915402271166b32fdf4ebdb708d58 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Sun, 29 Jan 2023 19:50:06 +0100 Subject: [PATCH] rpc,security: allow setting the tenant ID via gRPC metadata When running with shared-process multitenancy (with secondary tenant servers running in the same process as the KV node), we want to allow tenant servers to dial RPC connections with the same TLS client cert as the KV node. To make this possible, it becomes necessary for a RPC client to identify itself as a secondary tenant by another mean than the CN field in the TLS client cert. This patch does by introducing a new optional gRPC metadata header, "client-tenant". - When absent, we continue to use the tenant ID in the client cert as the claimed tenant identity of the client, as previously. - When present, it is used as the claimed tenant identity of the client. In that case, we allow two TLS situations: - either the client is _also_ using a client tenant cert, in which case we verify that the tenant ID in the cert matches the one in the metadata; - or, the client is using a regular client TLS cert, in which case we verify that it is using a 'root' or 'node' cert, since only these principals are allowed to perform RPCs in the first place. The authentication rules are summarized in the following table. This matrix was designed with the following properties: - *a rogue client cannot get more access by adding gRPC metadata than it would have if it didn't pass gRPC metadata at all*. This can be checked by observing that for a given TLS client cert, the addition of gRPC metadata always results in authn and authz rules that are at least as restrictive. - the gRPC metadata remains optional, so as to allow previous version SQL pods to continue to connect with just a valid TLS cert. ``` +------------------+---------------+-------------------------------------+--------------------------------------+ | | | Server is system tenant | Server is secondary tenant | +------------------+---------------+---------------------+---------------+-------------------------+------------+ | TLS client cert | gRPC metadata | Authn result | Authz rule | Authn result | Authz rule | +------------------+---------------+---------------------+---------------+-------------------------+------------+ | Tenant client | None | OK | tenant-filter | OK if client tenant | allow | | | | | | ID matches server | | | | | | | | | +------------------+ +---------------------+---------------+-------------------------+------------+ | `root` or `node` | | OK | allow | OK if user scope | allow | | client | | | | maches server tenant ID | | +------------------+ +---------------------+---------------+-------------------------+------------+ | other client | | deny | N/A | deny | N/A | +------------------+---------------+---------------------+---------------+-------------------------+------------+ | Tenant client | Client | OK if TLS tenant ID | tenant-filter | OK if TLS, MD and | allow | | | specified | matches MD | | server tenant IDs match | | +------------------+ tenant ID +---------------------+---------------+-------------------------+------------+ | `root` or `node` | | OK | tenant-filter | OK if MD and server | allow | | client | | | | tenant IDs match | | +------------------+ +---------------------+---------------+-------------------------+------------+ | other client | | deny | N/A | deny | N/A | +------------------+---------------+---------------------+---------------+-------------------------+------------+ ``` Release note: None --- pkg/rpc/auth.go | 292 +++++++++++++++++++++++++++++----------- pkg/rpc/auth_tenant.go | 6 +- pkg/rpc/auth_test.go | 58 ++++++-- pkg/rpc/context.go | 56 ++++++++ pkg/rpc/helpers_test.go | 11 +- pkg/server/status.go | 22 +++ 6 files changed, 356 insertions(+), 89 deletions(-) diff --git a/pkg/rpc/auth.go b/pkg/rpc/auth.go index 9ba4256cdfd8..3dc3b91f3804 100644 --- a/pkg/rpc/auth.go +++ b/pkg/rpc/auth.go @@ -12,11 +12,14 @@ package rpc import ( "context" + "crypto/x509" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" @@ -59,15 +62,30 @@ func (a kvAuth) unaryInterceptor( return handler(ctx, req) } - tenID, err := a.authenticate(ctx) + // Perform authentication and authz selection. + authnRes, authz, err := a.authenticateAndSelectAuthzRule(ctx) if err != nil { return nil, err } - ctx = contextWithClientTenant(ctx, tenID) - if tenID.IsSet() { - if err := a.tenant.authorize(tenID, info.FullMethod, req); err != nil { + + // Enhance the context if the peer is a tenant server. + switch ar := authnRes.(type) { + case authnSuccessPeerIsTenantServer: + ctx = contextWithClientTenant(ctx, roachpb.TenantID(ar)) + } + + // Handle authorization according to the selected authz method. + switch ar := authz.(type) { + case authzTenantServerToKVServer: + if err := a.tenant.authorize(roachpb.TenantID(ar), info.FullMethod, req); err != nil { return nil, err } + case authzTenantServerToTenantServer: + // Tenant servers can see all of each other's RPCs. + case authzPrivilegedPeerToServer: + // Privileged clients (root/node) can see all RPCs. + default: + return nil, errors.AssertionFailedf("unhandled case: %T", err) } return handler(ctx, req) } @@ -76,12 +94,22 @@ func (a kvAuth) streamInterceptor( srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler, ) error { ctx := ss.Context() - tenID, err := a.authenticate(ctx) + + // Perform authentication and authz selection. + authnRes, authz, err := a.authenticateAndSelectAuthzRule(ctx) if err != nil { return err } - ctx = contextWithClientTenant(ctx, tenID) - if tenID != (roachpb.TenantID{}) { + + // Enhance the context if the peer is a tenant server. + switch ar := authnRes.(type) { + case authnSuccessPeerIsTenantServer: + ctx = contextWithClientTenant(ctx, roachpb.TenantID(ar)) + } + + // Handle authorization according to the selected authz method. + switch ar := authz.(type) { + case authzTenantServerToKVServer: origSS := ss ss = &wrappedServerStream{ ServerStream: origSS, @@ -91,111 +119,221 @@ func (a kvAuth) streamInterceptor( return err } // 'm' is now populated and contains the request from the client. - return a.tenant.authorize(tenID, info.FullMethod, m) + return a.tenant.authorize(roachpb.TenantID(ar), info.FullMethod, m) }, } + case authzTenantServerToTenantServer: + // Tenant servers can see all of each other's RPCs. + case authzPrivilegedPeerToServer: + // Privileged clients (root/node) can see all RPCs. + default: + return errors.AssertionFailedf("unhandled case: %T", err) } return handler(srv, ss) } -// authenticate returns the client tenant ID of an RPC. An empty TenantID is -// returned if authorization should not be performed because the caller is -// allowed to perform any RPC. -func (a kvAuth) authenticate(ctx context.Context) (roachpb.TenantID, error) { - // Deal with local requests done through the internalClientAdapter. There's no - // TLS for these calls, so the regular authentication code path doesn't apply. - { - clientTenantID, localRequest := grpcutil.IsLocalRequestContext(ctx) - if localRequest { - if clientTenantID == roachpb.SystemTenantID { - // Bypass authentication check. - return roachpb.TenantID{}, nil - } - return clientTenantID, nil - } +func (a kvAuth) authenticateAndSelectAuthzRule( + ctx context.Context, +) (authnResult, requiredAuthzMethod, error) { + // Perform authentication. + authnRes, err := a.authenticate(ctx) + if err != nil { + return nil, nil, err + } + + // Select authorization rules suitable for the peer. + authz, err := a.selectAuthzMethod(ctx, authnRes) + if err != nil { + return nil, nil, err } + return authnRes, authz, nil +} + +// authnResult is a sum type that describes how RPC authentication has succeeded. +// This is used as input to selectAuthzMethod. +type authnResult interface { + authnResult() +} + +func getClientCert(ctx context.Context) (*x509.Certificate, error) { p, ok := peer.FromContext(ctx) if !ok { - return roachpb.TenantID{}, errTLSInfoMissing + return nil, errTLSInfoMissing } tlsInfo, ok := p.AuthInfo.(credentials.TLSInfo) if !ok || len(tlsInfo.State.PeerCertificates) == 0 { - return roachpb.TenantID{}, errTLSInfoMissing + return nil, errTLSInfoMissing } clientCert := tlsInfo.State.PeerCertificates[0] - if a.tenant.tenantID == roachpb.SystemTenantID { - // This node is a KV node. - // - // Is this a connection from a SQL tenant server? - if security.IsTenantCertificate(clientCert) { - // Incoming connection originating from a tenant SQL server, - // into a KV node. - // We extract the tenant ID to perform authorization - // of the RPC for this particular tenant. - return tenantFromCommonName(clientCert.Subject.CommonName) + return clientCert, nil +} + +// authnSuccessPeerIsTenantServer indicates authentication has +// succeeded, and the peer wishes to identify itself as a tenant +// server with this tenant ID. +type authnSuccessPeerIsTenantServer roachpb.TenantID + +// authnSuccessPeerIsPrivileged indicates authentication +// has succeeded, and the peer has used a root or node client cert. +type authnSuccessPeerIsPrivileged struct{} + +func (authnSuccessPeerIsTenantServer) authnResult() {} +func (authnSuccessPeerIsPrivileged) authnResult() {} + +// authenticate verifies the credentials of the client and performs +// some consistency check with the information provided. +func (a kvAuth) authenticate(ctx context.Context) (authnResult, error) { + // Do we have a wanted tenant ID from the request metadata? + var wantedTenantID roachpb.TenantID + + // Deal with local requests done through the internalClientAdapter. There's no + // TLS for these calls, so the regular authentication code path doesn't apply. + if clientTenantID, localRequest := grpcutil.IsLocalRequestContext(ctx); localRequest { + wantedTenantID = clientTenantID + + // Sanity check: verify that we do not also have gRPC network credentials + // in the context. This would indicate that metadata was improperly propagated. + maybeTid, err := getTenantIDFromNetworkCredentials(ctx) + if err != nil || maybeTid.IsSet() { + return nil, authErrorf("programming error: network credentials in internal adapter request (%v, %v)", maybeTid, err) } } else { - // This node is a SQL tenant server. - // - // Is this a connection from another SQL tenant server? + // TLS case. We will need to look at the TLS cert in any case, so + // extract it first. + clientCert, err := getClientCert(ctx) + if err != nil { + return nil, err + } + + // Did the client peer pass a tenant ID via the gRPC metadata? + maybeTid, err := getTenantIDFromNetworkCredentials(ctx) + if err != nil { + return nil, authErrorf("client provided invalid tenant ID: %v", err) + } else if maybeTid.IsSet() { + wantedTenantID = maybeTid + } + + // Did the client peer use a tenant client cert? if security.IsTenantCertificate(clientCert) { - // Incoming connection originating from a tenant SQL server. - tid, err := tenantFromCommonName(clientCert.Subject.CommonName) + // If the peer is using a client tenant cert, in any case we + // validate the tenant ID stored in the CN for correctness. + tlsID, err := tenantIDFromString(clientCert.Subject.CommonName, "Common Name (CN)") if err != nil { - return roachpb.TenantID{}, err + return nil, err } - // Verify that our peer is a service for the same tenant - // as ourselves (we don't want to allow tenant 123 to - // serve requests for a client coming from tenant 456). - if tid != a.tenant.tenantID { - return roachpb.TenantID{}, authErrorf("this tenant (%v) cannot serve requests from a server for tenant %v", a.tenant.tenantID, tid) + // If the peer is using a client tenant cert, either: + // - there was a wanted tenant ID in the metadata, in which case + // we verify the metadata ID is the same as the ID in the cert; + // or + // - there was no wanted ID in the metadata, in which case we use + // the tenant ID in the cert. + if wantedTenantID.IsSet() { + // Verify conformance. + if tlsID != wantedTenantID { + return nil, authErrorf( + "client wants to authenticate as tenant %v, but is using TLS cert for tenant %v", + wantedTenantID, tlsID) + } + } else { + // Use the ID in the cert as wanted tenant ID. + wantedTenantID = tlsID + } + } else { + // We are using TLS, but the peer is not using a client tenant + // cert. In that case, we only allow RPCs if the principal is + // 'node' or 'root' and the tenant scope in the cert matches + // this server (either the cert has scope "global" or its scope + // tenant ID matches our own). + // + // 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. + certUserScope, err := security.GetCertificateUserScope(clientCert) + if err != nil { + return nil, err + } + if err := checkRootOrNodeInScope(certUserScope, a.tenant.tenantID); err != nil { + return nil, err } - - // We return an unset tenant ID, to bypass authorization checks: - // the other server is able to use any of this server's RPCs. - return roachpb.TenantID{}, nil } } - // Here we handle the following cases: - // - // - incoming connection from a RPC admin client into either a KV - // node or a SQL server, using a valid root or node client cert. - // - incoming connections from another KV node into a KV node, using - // a node client cert. - // - calls coming through the gRPC gateway, from an HTTP client. The gRPC - // gateway uses a connection dialed as the node user. - // - // In both cases, we must check that the client cert is either root - // or node. We also need to check that the tenant scope for the cert - // is either the system tenant ID or matches the tenant ID of the server. - - // 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. - certUserScope, err := security.GetCertificateUserScope(clientCert) - if err != nil { - return roachpb.TenantID{}, err + if wantedTenantID.IsSet() { + // Consistency check: the peer wants to authenticate as another + // tenant server, verify that the peer is a service for the same + // tenant as ourselves (we don't want to allow tenant 123 to serve + // requests for a client coming from tenant 456) or we are the system tenant. + if !(a.tenant.tenantID.IsSystem() || wantedTenantID == a.tenant.tenantID) { + log.Ops.Infof(ctx, "rejected incoming request from tenant %d (misconfiguration?)", wantedTenantID) + return nil, authErrorf("client tenant identity (%v) does not match server", wantedTenantID) + } + return authnSuccessPeerIsTenantServer(wantedTenantID), nil } + return authnSuccessPeerIsPrivileged{}, nil +} - // Confirm that the user scope is node/root. Otherwise, return an authentication error. - if err := checkRootOrNodeInScope(certUserScope, a.tenant.tenantID); err != nil { - return roachpb.TenantID{}, err - } +// requiredAuthzMethod is a sum type that describes which authorization +// rules to use to determine whether a RPC is allowed or not. +type requiredAuthzMethod interface { + rpcAuthzMethod() +} + +// Tenant server connecting to KV node. +type authzTenantServerToKVServer roachpb.TenantID + +// Tenant server connecting to another tenant server. +type authzTenantServerToTenantServer struct{} + +// External client connecting to tenant server or KV node using 'root' +// or 'node' client cert; or KV node connecting to other KV node. +type authzPrivilegedPeerToServer struct{} + +func (authzTenantServerToKVServer) rpcAuthzMethod() {} +func (authzTenantServerToTenantServer) rpcAuthzMethod() {} +func (authzPrivilegedPeerToServer) rpcAuthzMethod() {} - // User is node/root user authorized for this tenant, return success. - return roachpb.TenantID{}, nil +// selectAuthzMethod selects the authorization rule to use for the +// given authentication event. +func (a kvAuth) selectAuthzMethod( + ctx context.Context, ar authnResult, +) (requiredAuthzMethod, error) { + switch res := ar.(type) { + case authnSuccessPeerIsTenantServer: + // The client is a tenant server. We have two possible cases: + // - tenant server to KV node. + // - tenant server to another tenant server. + if a.tenant.tenantID == roachpb.SystemTenantID { + return authzTenantServerToKVServer(res), nil + } + return authzTenantServerToTenantServer{}, nil + + case authnSuccessPeerIsPrivileged: + // Here are the remaining cases: + // + // - incoming connection from a RPC admin client into either a KV + // node or a SQL server, using a valid root or node client cert. + // - incoming connections from another KV node into a KV node, using + // a node client cert. + // - calls coming through the gRPC gateway, from an HTTP client. The gRPC + // gateway uses a connection dialed as the node user. + // + // In all these cases, the RPC request is authorized. + return authzPrivilegedPeerToServer{}, nil + + default: + return nil, errors.AssertionFailedf("programming error: unhandled case %T", ar) + } } // checkRootOrNodeInScope checks that the root or node principals are // present in the cert user scopes. func checkRootOrNodeInScope( certUserScope []security.CertificateUserScope, serverTenantID roachpb.TenantID, -) (err error) { +) error { for _, scope := range certUserScope { // Only consider global scopes or scopes that match this server. if !(scope.Global || scope.TenantID == serverTenantID) { diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index f577c3e14719..9e3365ca6f8d 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -30,13 +30,13 @@ type tenantAuthorizer struct { tenantID roachpb.TenantID } -func tenantFromCommonName(commonName string) (roachpb.TenantID, error) { +func tenantIDFromString(commonName, field string) (roachpb.TenantID, error) { 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) + return roachpb.TenantID{}, authErrorf("could not parse tenant ID from %s: %s", field, 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.TenantID{}, authErrorf("invalid tenant ID %d in %s", tenID, field) } return roachpb.MustMakeTenantID(tenID), nil } diff --git a/pkg/rpc/auth_test.go b/pkg/rpc/auth_test.go index 4ac3cd28465a..b6188ced4a55 100644 --- a/pkg/rpc/auth_test.go +++ b/pkg/rpc/auth_test.go @@ -97,12 +97,13 @@ func TestAuthenticateTenant(t *testing.T) { stid := roachpb.SystemTenantID tenTen := roachpb.MustMakeTenantID(10) for _, tc := range []struct { - systemID roachpb.TenantID - ous []string - commonName string - expTenID roachpb.TenantID - expErr string - tenantScope uint64 + systemID roachpb.TenantID + ous []string + commonName string + expTenID roachpb.TenantID + expErr string + tenantScope uint64 + clientTenantInMD string }{ {systemID: stid, ous: correctOU, commonName: "10", expTenID: tenTen}, {systemID: stid, ous: correctOU, commonName: roachpb.MinTenantID.String(), expTenID: roachpb.MinTenantID}, @@ -123,12 +124,48 @@ func TestAuthenticateTenant(t *testing.T) { {systemID: stid, ous: nil, commonName: "root", tenantScope: 10, expErr: `need root or node client cert to perform RPCs on this server \(this is tenant system; cert is valid for "root" on tenant 10\)`}, {systemID: tenTen, ous: correctOU, commonName: "10", expTenID: roachpb.TenantID{}}, - {systemID: tenTen, ous: correctOU, commonName: "123", expErr: `this tenant \(10\) cannot serve requests from a server for tenant 123`}, + {systemID: tenTen, ous: correctOU, commonName: "123", expErr: `client tenant identity \(123\) does not match server`}, {systemID: tenTen, ous: correctOU, commonName: "1", expErr: `invalid tenant ID 1 in Common Name \(CN\)`}, {systemID: tenTen, ous: nil, commonName: "root"}, {systemID: tenTen, ous: nil, commonName: "node"}, + + // Passing a client ID in metadata instead of relying only on the TLS cert. + {clientTenantInMD: "invalid", expErr: `could not parse tenant ID from gRPC metadata`}, + {clientTenantInMD: "1", expErr: `invalid tenant ID 1 in gRPC metadata`}, + {clientTenantInMD: "-1", expErr: `could not parse tenant ID from gRPC metadata`}, + + // tenant ID in MD matches that in client cert. + // Server is KV node: expect tenant authorization. + {clientTenantInMD: "10", + systemID: stid, ous: correctOU, commonName: "10", expTenID: tenTen}, + // tenant ID in MD doesn't match that in client cert. + {clientTenantInMD: "10", + systemID: stid, ous: correctOU, commonName: "123", + expErr: `client wants to authenticate as tenant 10, but is using TLS cert for tenant 123`}, + // tenant ID present in MD, but not in client cert. However, + // client cert is valid. Use MD tenant ID. + // Server is KV node: expect tenant authorization. + {clientTenantInMD: "10", + systemID: stid, ous: nil, commonName: "root", expTenID: tenTen}, + // tenant ID present in MD, but not in client cert. However, + // client cert is valid. Use MD tenant ID. + // Server is KV node: expect tenant authorization. + {clientTenantInMD: "10", + systemID: stid, ous: nil, commonName: "node", expTenID: tenTen}, + // tenant ID present in MD, but not in client cert. However, + // client cert is valid. Use MD tenant ID. + // Server is secondary tenant: do not do additional tenant authorization. + {clientTenantInMD: "10", + systemID: tenTen, ous: nil, commonName: "root", expTenID: roachpb.TenantID{}}, + {clientTenantInMD: "10", + systemID: tenTen, ous: nil, commonName: "node", expTenID: roachpb.TenantID{}}, + // tenant ID present in MD, but not in client cert. Use MD tenant ID. + // Server tenant ID does not match client tenant ID. + {clientTenantInMD: "123", + systemID: tenTen, ous: nil, commonName: "root", + expErr: `client tenant identity \(123\) does not match server`}, } { - t.Run(fmt.Sprintf("from %v to %v", tc.commonName, tc.systemID), func(t *testing.T) { + t.Run(fmt.Sprintf("from %v to %v (md %q)", tc.commonName, tc.systemID, tc.clientTenantInMD), func(t *testing.T) { cert := &x509.Certificate{ Subject: pkix.Name{ CommonName: tc.commonName, @@ -150,6 +187,11 @@ func TestAuthenticateTenant(t *testing.T) { p := peer.Peer{AuthInfo: tlsInfo} ctx := peer.NewContext(context.Background(), &p) + if tc.clientTenantInMD != "" { + md := metadata.MD{rpc.ClientTIDMetadataHeaderKey: []string{tc.clientTenantInMD}} + ctx = metadata.NewIncomingContext(ctx, md) + } + tenID, err := rpc.TestingAuthenticateTenant(ctx, tc.systemID) if tc.expErr == "" { diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index eb12d3c8da79..09e7d07d3fab 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -419,6 +419,9 @@ type Context struct { // The loopbackDialFn fits under that common case by transporting // the gRPC protocol over an in-memory pipe. loopbackDialFn func(context.Context) (net.Conn, error) + + // clientCreds is used to pass additional headers to called RPCs. + clientCreds credentials.PerRPCCredentials } // SetLoopbackDialer configures the loopback dialer function. @@ -613,6 +616,10 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { logClosingConnEvery: log.Every(time.Second), } + if !opts.TenantID.IsSystem() { + rpcCtx.clientCreds = newTenantClientCreds(opts.TenantID) + } + if opts.Knobs.NoLoopbackDialer { // The test has decided it doesn't need/want a loopback dialer. // Ensure we still have a working dial function in that case. @@ -666,6 +673,51 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { return rpcCtx } +// tenantClientCred is responsible for passing the tenant ID as +// medatada header to called RPCs. This makes it possible to pass the +// tenant ID even when using a different TLS cert than the "tenant +// client cert". +type tenantClientCred struct { + md map[string]string +} + +// ClientTIDMetadataHeaderKey is the gRPC metadata key that indicates +// which tenant ID the client is intending to connect as (originating +// tenant identity). +// +// This is used instead of the cert CN field when connecting with a +// TLS client cert that is not marked as special "tenant client cert" +// via the "Tenants" string in the OU field. +const ClientTIDMetadataHeaderKey = "client-tid" + +func newTenantClientCreds(tid roachpb.TenantID) credentials.PerRPCCredentials { + return &tenantClientCred{ + md: map[string]string{ + ClientTIDMetadataHeaderKey: fmt.Sprint(tid), + }, + } +} + +func getTenantIDFromNetworkCredentials(ctx context.Context) (roachpb.TenantID, error) { + val := metadata.ValueFromIncomingContext(ctx, ClientTIDMetadataHeaderKey) + if len(val) == 0 { + return roachpb.TenantID{}, nil + } + return tenantIDFromString(val[0], "gRPC metadata") +} + +// GetRequestMetadata implements the (grpc) +// credentials.PerRPCCredentials interface. +func (tcc *tenantClientCred) GetRequestMetadata( + ctx context.Context, uri ...string, +) (map[string]string, error) { + return tcc.md, nil +} + +// RequireTransportSecurity implements the (grpc) +// credentials.PerRPCCredentials interface. +func (tcc *tenantClientCred) RequireTransportSecurity() bool { return false } + // ClusterName retrieves the configured cluster name. func (rpcCtx *Context) ClusterName() string { if rpcCtx == nil { @@ -1691,6 +1743,10 @@ func (rpcCtx *Context) dialOptsCommon( grpc.MaxCallSendMsgSize(math.MaxInt32), )} + if rpcCtx.clientCreds != nil { + dialOpts = append(dialOpts, grpc.WithPerRPCCredentials(rpcCtx.clientCreds)) + } + // We throw this one in for good measure, but it only disables the retries // for RPCs that were already pending (which are opt in anyway, and we don't // opt in). It doesn't disable what gRPC calls "transparent retries" (RPC diff --git a/pkg/rpc/helpers_test.go b/pkg/rpc/helpers_test.go index 39bb2dfb2753..e7b79e7a4ae9 100644 --- a/pkg/rpc/helpers_test.go +++ b/pkg/rpc/helpers_test.go @@ -39,7 +39,16 @@ func TestingNewWrappedServerStream( func TestingAuthenticateTenant( ctx context.Context, serverTenantID roachpb.TenantID, ) (roachpb.TenantID, error) { - return kvAuth{tenant: tenantAuthorizer{tenantID: serverTenantID}}.authenticate(ctx) + _, authz, err := kvAuth{tenant: tenantAuthorizer{tenantID: serverTenantID}}.authenticateAndSelectAuthzRule(ctx) + if err != nil { + return roachpb.TenantID{}, err + } + switch z := authz.(type) { + case authzTenantServerToKVServer: + return roachpb.TenantID(z), nil + default: + return roachpb.TenantID{}, nil + } } // TestingAuthorizeTenantRequest performs authorization of a tenant request diff --git a/pkg/server/status.go b/pkg/server/status.go index 60fe7639b103..0350f9b44148 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -135,6 +135,28 @@ type metricMarshaler interface { func propagateGatewayMetadata(ctx context.Context) context.Context { if md, ok := metadata.FromIncomingContext(ctx); ok { + // Here we erase the tenant ID information from the RPC client. + // + // To understand why this works and is important, we need to + // consider two cases: + // - SQL performs RPC to KV, KV performs RPC to another KV node. + // (a.k.a. "node-to-node RPCs") + // - external client performs RPC to SQL, SQL performs RPC to + // another SQL server. (a.k.a. "pod-to-pod RPCs"). + // + // In the first case, it is very necessary to clear the tenant ID. + // When SQL for tenant 10 performs a RPC to KV n1, and KV n1 + // performs a gateway forward to KV n2, we don't want n1 to + // authenticate itself as "tenant 10" to n2. At this point, we are + // doing work "on behalf of KV" and the tenant information must + // not be used any more. + // + // In the second case, it is not strictly necessary to clear the + // tenant ID here -- the next SQL server could use it -- however + // it does not hurt: the RPC dial function will re-add it anyway + // via the PerRPCCredentials. So we delete it anyway to + // keep the code simple. + delete(md, rpc.ClientTIDMetadataHeaderKey) return metadata.NewOutgoingContext(ctx, md) } return ctx