From 26099239b65107a5c334cd914784464785363bc2 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Tue, 31 Jan 2023 23:46:48 +0100 Subject: [PATCH] rpc: improve the authn/authz control flow This patch more clearly separate authentication from authorization, and makes it clear when the flow involves a tenant ID. Release note: None --- pkg/rpc/auth.go | 264 ++++++++++++++++++++++++++++------------ pkg/rpc/auth_tenant.go | 6 +- pkg/rpc/auth_test.go | 2 +- pkg/rpc/helpers_test.go | 11 +- 4 files changed, 201 insertions(+), 82 deletions(-) diff --git a/pkg/rpc/auth.go b/pkg/rpc/auth.go index 9ba4256cdfd8..f27f28e46774 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,193 @@ 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 { + if !clientTenantID.IsSystem() { + wantedTenantID = clientTenantID } } 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 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) + // 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{} - // User is node/root user authorized for this tenant, return success. - return roachpb.TenantID{}, nil +// 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() {} + +// 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..047dacfccccc 100644 --- a/pkg/rpc/auth_test.go +++ b/pkg/rpc/auth_test.go @@ -123,7 +123,7 @@ 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"}, 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