Skip to content

Commit

Permalink
rpc,security: allow setting the tenant ID via gRPC metadata
Browse files Browse the repository at this point in the history
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
  • Loading branch information
knz committed Jan 30, 2023
1 parent 111432e commit b4f4d6b
Show file tree
Hide file tree
Showing 6 changed files with 356 additions and 89 deletions.
292 changes: 215 additions & 77 deletions pkg/rpc/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
}
Expand All @@ -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,
Expand All @@ -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) {
Expand Down
6 changes: 3 additions & 3 deletions pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
Loading

0 comments on commit b4f4d6b

Please sign in to comment.