Skip to content

Commit

Permalink
cert: Use tenant scoped client cert for auth.
Browse files Browse the repository at this point in the history
This PR extends the TLS code to use a tenant scoped
client cert to authenticate a client for specific tenant.

Release note: None
  • Loading branch information
rimadeodhar committed Mar 30, 2022
1 parent 0f79e38 commit 74f804b
Show file tree
Hide file tree
Showing 14 changed files with 153 additions and 33 deletions.
3 changes: 1 addition & 2 deletions pkg/cli/client_url.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (
"path/filepath"

"github.com/cockroachdb/cockroach/pkg/cli/cliflags"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/pgurl"
Expand Down Expand Up @@ -360,7 +359,7 @@ func (cliCtx *cliContext) makeClientConnURL() (*pgurl.URL, error) {
userName = security.RootUserName()
}

sCtx := rpc.MakeSecurityContext(cliCtx.Config, security.CommandTLSSettings{}, roachpb.SystemTenantID)
sCtx := rpc.MakeSecurityContext(cliCtx.Config, security.CommandTLSSettings{}, cliCtx.tenantID)
if err := sCtx.LoadSecurityOptions(purl, userName); err != nil {
return nil, err
}
Expand Down
6 changes: 6 additions & 0 deletions pkg/cli/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package cli

import (
"context"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"io"
"os"
"strconv"
Expand Down Expand Up @@ -200,6 +201,10 @@ type cliContext struct {

// For `cockroach version --build-tag`.
showVersionUsingOnlyBuildTag bool

// tenantID indicates the tenant to run the CLI utility against.
// Default value is the system tenant.
tenantID roachpb.TenantID
}

// cliCtx captures the command-line parameters common to most CLI utilities.
Expand Down Expand Up @@ -233,6 +238,7 @@ func setCliContextDefaults() {
// TODO(knz): Deprecated in v21.1. Remove this.
cliCtx.deprecatedLogOverrides.reset()
cliCtx.showVersionUsingOnlyBuildTag = false
cliCtx.tenantID = roachpb.SystemTenantID
}

// sqlConnContext captures the connection configuration for all SQL
Expand Down
32 changes: 17 additions & 15 deletions pkg/rpc/pg.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package rpc

import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"net/url"

"github.com/cockroachdb/cockroach/pkg/base"
Expand Down Expand Up @@ -77,34 +78,28 @@ func (ctx *SecurityContext) LoadSecurityOptions(u *pgurl.URL, username security.
// (Re)populate the transport information.
u.WithTransport(pgurl.TransportTLS(tlsMode, caCertPath))

var missing bool // certs found on file system?
loader := security.GetAssetLoader()

// Fetch client certs, but don't fail if they're absent, we may be
// using a password.
certPath := ctx.ClientCertPath(username)
keyPath := ctx.ClientKeyPath(username)
_, err1 := loader.Stat(certPath)
_, err2 := loader.Stat(keyPath)
if err1 != nil || err2 != nil {
missing = true
certsAvailable := checkCertAndKeyAvailable(certPath, keyPath)
if !certsAvailable && ctx.tenID != roachpb.SystemTenantID {
// Fetch tenant scoped client certs, if any.
certPath = ctx.ClientForTenantCertPath(username, ctx.tenID.String())
keyPath = ctx.ClientForTenantKeyPath(username, ctx.tenID.String())
certsAvailable = checkCertAndKeyAvailable(certPath, keyPath)
}
// If the command specifies user node, and we did not find
// client.node.crt, try with just node.crt.
if missing && username.IsNodeUser() {
missing = false
if !certsAvailable && username.IsNodeUser() {
certPath = ctx.NodeCertPath()
keyPath = ctx.NodeKeyPath()
_, err1 = loader.Stat(certPath)
_, err2 = loader.Stat(keyPath)
if err1 != nil || err2 != nil {
missing = true
}
certsAvailable = checkCertAndKeyAvailable(certPath, keyPath)
}

// If we found some certs, add them to the URL authentication
// method.
if !missing {
if certsAvailable {
pwEnabled, hasPw, pwd := u.GetAuthnPassword()
if !pwEnabled {
u.WithAuthn(pgurl.AuthnClientCert(certPath, keyPath))
Expand All @@ -130,3 +125,10 @@ func (ctx *SecurityContext) PGURL(user *url.Userinfo) (*pgurl.URL, error) {
}
return u, nil
}

func checkCertAndKeyAvailable(certPath string, keyPath string) bool {
loader := security.GetAssetLoader()
_, err1 := loader.Stat(certPath)
_, err2 := loader.Stat(keyPath)
return err1 == nil && err2 == nil
}
1 change: 1 addition & 0 deletions pkg/security/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/roachpb",
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
Expand Down
53 changes: 51 additions & 2 deletions pkg/security/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@ import (
"context"
"crypto/tls"
"crypto/x509"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"strconv"
"strings"

"github.com/cockroachdb/cockroach/pkg/util/syncutil"
Expand Down Expand Up @@ -107,15 +109,19 @@ func Contains(sl []string, s string) bool {

// UserAuthCertHook builds an authentication hook based on the security
// mode and client certificate.
func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAuthHook, error) {
func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState, tenantID roachpb.TenantID) (UserAuthHook, error) {
var certUsers []string

var certTenantID roachpb.TenantID
if !insecureMode {
var err error
certUsers, err = GetCertificateUsers(tlsState)
if err != nil {
return nil, err
}
certTenantID, err = GetTenantIfTenantScopedClientCert(tlsState)
if err != nil {
return nil, err
}
}

return func(ctx context.Context, systemIdentity SQLUsername, clientConnection bool) error {
Expand All @@ -140,6 +146,20 @@ func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAut
return errors.Errorf("using tenant client certificate as user certificate is not allowed")
}

// If the current server is a non system tenant SQL server, we should use a tenant scoped
// client certificate.
// TODO(rima): Should we enforce always using tenant scoped client cert for non-system tenants?
if tenantID != roachpb.SystemTenantID {
// Enforce that the tenant ID *and* user matches the certificate
if tenantID != certTenantID {
return errors.Errorf("certificate is for tenant ID %s, but current tenant ID is %s", certTenantID, tenantID)
}
if !Contains(certUsers, systemIdentity.Normalized()) {
return errors.Errorf("requested user is %s, but certificate is for %s", systemIdentity, certUsers)
}
}
// TODO(rima): If we always want to enforce using tenant scoped client cert for non system tenants, we will
// need to put the below in an else block.
// The client certificate user must match the requested user.
if !Contains(certUsers, systemIdentity.Normalized()) {
return errors.Errorf("requested user is %s, but certificate is for %s", systemIdentity, certUsers)
Expand Down Expand Up @@ -195,3 +215,32 @@ func UserAuthPasswordHook(
func NewErrPasswordUserAuthFailed(username SQLUsername) error {
return errors.Newf("password authentication failed for user %s", username)
}

func GetTenantIfTenantScopedClientCert(tlsState *tls.ConnectionState) (roachpb.TenantID, error) {
if tlsState == nil {
return roachpb.TenantID{}, errors.Errorf("request is not using TLS")
}
if len(tlsState.PeerCertificates) == 0 {
return roachpb.TenantID{}, errors.Errorf("no client certificates in request")
}
// The go server handshake code verifies the first certificate, using
// any following certificates as intermediates. See:
// https://github.com/golang/go/blob/go1.8.1/src/crypto/tls/handshake_server.go#L723:L742
peerCert := tlsState.PeerCertificates[0]
uris := peerCert.URIs
var tenantID uint64
var err error
for _, uri := range uris {
if uri.Host == "tenant" {
tenantInfo := strings.TrimPrefix(uri.Path, "/")
tenantID, err = strconv.ParseUint(tenantInfo, 10, 64)
if err != nil {
return roachpb.TenantID{}, errors.Wrapf(err, "tenant ID: %s contained in cert is invalid", tenantInfo)
}
return roachpb.MakeTenantID(tenantID), nil
}
}

// No tenant info contained within cert, return default system tenant
return roachpb.SystemTenantID, nil
}
3 changes: 2 additions & 1 deletion pkg/security/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"crypto/tls"
"crypto/x509"
"crypto/x509/pkix"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"strings"
"testing"

Expand Down Expand Up @@ -217,7 +218,7 @@ func TestAuthenticationHook(t *testing.T) {
if err != nil {
t.Fatal(err)
}
hook, err := security.UserAuthCertHook(tc.insecure, makeFakeTLSState(tc.tlsSpec))
hook, err := security.UserAuthCertHook(tc.insecure, makeFakeTLSState(tc.tlsSpec), roachpb.SystemTenantID)
if (err == nil) != tc.buildHookSuccess {
t.Fatalf("expected success=%t, got err=%v", tc.buildHookSuccess, err)
}
Expand Down
34 changes: 32 additions & 2 deletions pkg/security/certificate_loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"os"
"path/filepath"
"runtime"
"strconv"
"strings"
"time"

Expand Down Expand Up @@ -93,6 +94,9 @@ const (
UIPem
// ClientPem describes a client certificate.
ClientPem
// TenantScopedClientPem describes a tenant scoped client certificate.
// This certificate can only be used to authenticate a client for a specific tenant.
TenantScopedClientPem
// TenantPem describes a SQL tenant client certificate.
TenantPem
// TenantSigningPem describes a SQL tenant signing certificate.
Expand Down Expand Up @@ -223,11 +227,20 @@ func CertInfoFromFilename(filename string) (*CertInfo, error) {
return nil, errors.Errorf("UI certificate filename should match ui%s", certExtension)
}
case `client`:
fileUsage = ClientPem
// This could be a client certificate or a tenant scoped client certificate.
if strings.Contains(filename, "tenant") {
fileUsage = TenantScopedClientPem
} else {
fileUsage = ClientPem
}
// Strip prefix and suffix and re-join middle parts.
name = strings.Join(parts[1:numParts-1], `.`)
if len(name) == 0 {
return nil, errors.Errorf("client certificate filename should match client.<user>%s", certExtension)
if fileUsage == ClientPem {
return nil, errors.Errorf("client certificate filename should match client.<user>%s", certExtension)
} else {
return nil, errors.Errorf("tenant scoped client certificate filename should match client.<user>.tenant-<tenant_id>%s", certExtension)
}
}
case `client-tenant`:
fileUsage = TenantPem
Expand Down Expand Up @@ -506,3 +519,20 @@ func validateCockroachCertificate(ci *CertInfo, cert *x509.Certificate) error {
}
return nil
}

func extractTenantAndUserFromCertName(filename string) (SQLUsername, uint64, error) {
parts := strings.Split(filename, ".")
if len(parts) != 2 {
return SQLUsername{}, 0, errors.Errorf("expected tenant scoped cert name format is <user>.tenant-<tenant-id>")
}
tenantInfo := strings.Split(parts[1], "-")
if len(tenantInfo) != 2 {
return SQLUsername{}, 0, errors.Errorf("expected tenant ID format within tenant scoped client cert is tenant-<tenant-id>")
}
tenantID, err := strconv.ParseUint(tenantInfo[1], 10, 64)
if err != nil {
return SQLUsername{}, 0, errors.Errorf("invalid tenant id %s", tenantInfo[1])
}
username := MakeSQLUsernameFromPreNormalizedString(parts[0])
return username, tenantID, nil
}
25 changes: 18 additions & 7 deletions pkg/security/certificate_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,13 +117,14 @@ type CertificateManager struct {
initialized bool

// Set of certs. These are swapped in during Load(), and never mutated afterwards.
caCert *CertInfo // default CA certificate
clientCACert *CertInfo // optional: certificate to verify client certificates
uiCACert *CertInfo // optional: certificate to verify UI certificates
nodeCert *CertInfo // certificate for nodes (always server cert, sometimes client cert)
nodeClientCert *CertInfo // optional: client certificate for 'node' user. Also included in 'clientCerts'
uiCert *CertInfo // optional: server certificate for the admin UI.
clientCerts map[SQLUsername]*CertInfo
caCert *CertInfo // default CA certificate
clientCACert *CertInfo // optional: certificate to verify client certificates
uiCACert *CertInfo // optional: certificate to verify UI certificates
nodeCert *CertInfo // certificate for nodes (always server cert, sometimes client cert)
nodeClientCert *CertInfo // optional: client certificate for 'node' user. Also included in 'clientCerts'
uiCert *CertInfo // optional: server certificate for the admin UI.
clientCerts map[SQLUsername]*CertInfo
tenantScopedClientCerts map[SQLUsername]*CertInfo

// Certs only used with multi-tenancy.
tenantCACert, tenantCert, tenantSigningCert *CertInfo
Expand Down Expand Up @@ -643,6 +644,7 @@ func (cm *CertificateManager) LoadCertificates() error {
var caCert, clientCACert, uiCACert, nodeCert, uiCert, nodeClientCert *CertInfo
var tenantCACert, tenantCert, tenantSigningCert *CertInfo
clientCerts := make(map[SQLUsername]*CertInfo)
tenantScopedClientCerts := make(map[SQLUsername]*CertInfo)
for _, ci := range cl.Certificates() {
switch ci.FileUsage {
case CAPem:
Expand Down Expand Up @@ -685,6 +687,14 @@ func (cm *CertificateManager) LoadCertificates() error {
if username.IsNodeUser() {
nodeClientCert = ci
}
case TenantScopedClientPem:
username, tenantID, err := extractTenantAndUserFromCertName(ci.Name)
if err != nil {
return errors.Wrapf(err, "invalid tenant scoped client cert name %s", ci.Name)
}
if cm.tenantIdentifier == tenantID {
tenantScopedClientCerts[username] = ci
}
default:
return errors.Errorf("unsupported certificate %v", ci.Filename)
}
Expand Down Expand Up @@ -742,6 +752,7 @@ func (cm *CertificateManager) LoadCertificates() error {
cm.nodeClientCert = nodeClientCert
cm.uiCert = uiCert
cm.clientCerts = clientCerts
cm.tenantScopedClientCerts = tenantScopedClientCerts

cm.initialized = true

Expand Down
1 change: 1 addition & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -833,6 +833,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
rootSQLMemoryMonitor,
cfg.HistogramWindowInterval(),
execCfg,
cfg.TenantID,
)

distSQLServer.ServerConfig.SQLStatsController = pgServer.SQLServer.GetSQLStatsController()
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/pgwire/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_library(
"//pkg/base",
"//pkg/clusterversion",
"//pkg/col/coldata",
"//pkg/roachpb",
"//pkg/security",
"//pkg/security/sessionrevival",
"//pkg/server/serverpb",
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/pgwire/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"crypto/tls"
"fmt"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"net"
"strings"

Expand Down Expand Up @@ -78,6 +79,7 @@ type authOptions struct {
// testingAuthHook, if provided, replaces the logic in
// handleAuthentication().
testingAuthHook func(ctx context.Context) error
tenantID roachpb.TenantID
}

// handleAuthentication checks the connection's user. Errors are sent to the
Expand Down Expand Up @@ -109,7 +111,7 @@ func (c *conn) handleAuthentication(
// Populate the AuthMethod with per-connection information so that it
// can compose the next layer of behaviors that we're going to apply
// to the incoming connection.
behaviors, err := authMethod(ctx, ac, tlsState, execCfg, hbaEntry, authOpt.identMap)
behaviors, err := authMethod(ctx, ac, tlsState, execCfg, hbaEntry, authOpt.identMap, authOpt.tenantID)
connClose = behaviors.ConnClose
if err != nil {
ac.LogAuthFailed(ctx, eventpb.AuthFailReason_UNKNOWN, err)
Expand Down
Loading

0 comments on commit 74f804b

Please sign in to comment.