Skip to content

Commit

Permalink
rpc,security: use the tenant client cert for pod-pod communication
Browse files Browse the repository at this point in the history
As of this patch, we have the following file usage:

- KV nodes on host cluster:
  - ui.crt (optional):
    - used as server cert for HTTP
  - ui-ca.crt (optional):
    - used in unit tests to verify the server's identity for HTTP conns
  - node.crt:
    - used as client cert for node-to-node comms
    - used as server cert for node-to-node comms
    - used as server cert for SQL clients
    - used as server cert for incoming conns from SQL tenant servers
    - used as server cert for HTTP, if ui.crt doesn't exist
  - tenant-client-ca.crt (optional):
    - used to verify certificates from SQL tenant servers connecting as clients
  - client-ca.crt (optional);
    - used to verify client certs for SQL clients
    - used to verify client certs for SQL tenant servers, if tenant-client-ca.crt doesn't exist
  - ca.crt:
    - used to verify other node client certs for node-to-node comms
    - used in unit tests to verify the server's identity for SQL and RPC conns
    - used to verify client certs for SQL clients, if client-ca.crt doesn't exist
    - used to verify client certs for SQL tenant servers, if neither tenant-client.ca.crt nor client-ca.crt exist

- SQL servers:
  - ui.crt (optional):
    - used as server cert for HTTP
  - ui-ca.crt (optional):
    - used in unit tests to verify the server's identity for HTTP conns
  - client-tenant.NN.crt:
    - used as client cert for node-to-node comms (SQL server to SQL server)
    - used as server cert for node-to-node comms (SQL server to SQL server)
    - used as client cert for conns to KV nodes
    - used as server cert for SQL clients
    - used as server cert for HTTP, if ui.crt doesn't exist
  - tenant-client-ca.crt (optional):
    - used to verify certs from other SQL tenant servers
  - client-ca.crt (optional);
    - used to verify client certs for SQL clients
    - used to verify client certs for SQL tenant servers, if tenant-client-ca.crt doesn't exist
  - ca.crt:
    - used to verify other SQL server certs for node-to-node comms, if tenant-client-ca.crt doens't exist
    - used to verify client certs for SQL clients, if client-ca.crt doesn't exist
    - used to verify client certs for SQL tenant servers, if neither tenant-client.ca.crt nor client-ca.crt exist
    - used in unit tests to verify the server's identity for SQL and  RPC conns

Release note (security update): Multitenant SQL servers now reuse
the tenant client certificate (`client-tenant.NN.crt`) for SQL-to-SQL
communication. Existing deployments must regenerate the certificates
with dual purpose (client and server authentication).
  • Loading branch information
knz committed Oct 7, 2021
1 parent 617beac commit 406b396
Show file tree
Hide file tree
Showing 46 changed files with 758 additions and 610 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/serverccl/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

Expand All @@ -28,6 +29,7 @@ var adminPrefix = "/_admin/v1/"
// that we see all zone configs (#27718).
func TestAdminAPIDataDistributionPartitioning(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{})
defer testCluster.Stopper().Stop(context.Background())
Expand Down Expand Up @@ -80,6 +82,7 @@ func TestAdminAPIDataDistributionPartitioning(t *testing.T) {
// TestAdminAPIChartCatalog verifies that an error doesn't happen.
func TestAdminAPIChartCatalog(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{})
defer testCluster.Stopper().Stop(context.Background())
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/serverccl/role_authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,15 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/stretchr/testify/require"
"golang.org/x/crypto/bcrypt"
)

func TestVerifyPassword(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/serverccl/server_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,14 +141,15 @@ func TestTenantHTTP(t *testing.T) {
tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{})
defer tc.Stopper().Stop(ctx)

httpClient, err := tc.Server(0).RPCContext().GetHTTPClient()
require.NoError(t, err)

tenant, err := tc.Server(0).StartTenant(ctx,
base.TestTenantArgs{
TenantID: serverutils.TestTenantID(),
})
require.NoError(t, err)

httpClient, err := tenant.RPCContext().GetHTTPClient()
require.NoError(t, err)

t.Run("prometheus", func(t *testing.T) {
resp, err := httpClient.Get("https://" + tenant.HTTPAddr() + "/_status/vars")
defer http.DefaultClient.CloseIdleConnections()
Expand Down
8 changes: 5 additions & 3 deletions pkg/ccl/serverccl/tenant_grpc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,9 +45,6 @@ func TestTenantGRPCServices(t *testing.T) {

server := testCluster.Server(0)

httpClient, err := server.RPCContext().GetHTTPClient()
require.NoError(t, err)

tenantID := serverutils.TestTenantID()
tenant, connTenant := serverutils.StartTenant(t, server, base.TestTenantArgs{
TenantID: tenantID,
Expand All @@ -59,6 +56,8 @@ func TestTenantGRPCServices(t *testing.T) {
})
defer connTenant.Close()

t.Logf("subtests starting")

t.Run("gRPC is running", func(t *testing.T) {
grpcAddr := tenant.SQLAddr()
rpcCtx := tenant.RPCContext()
Expand All @@ -74,6 +73,9 @@ func TestTenantGRPCServices(t *testing.T) {
require.NotEmpty(t, resp.Statements)
})

httpClient, err := tenant.RPCContext().GetHTTPClient()
require.NoError(t, err)

t.Run("gRPC Gateway is running", func(t *testing.T) {
resp, err := httpClient.Get("https://" + tenant.HTTPAddr() + "/_status/statements")
defer http.DefaultClient.CloseIdleConnections()
Expand Down
51 changes: 29 additions & 22 deletions pkg/ccl/serverccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,8 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
err = serverutils.GetJSONProto(nonTenant, path, &nonTenantCombinedStats)
require.NoError(t, err)

checkStatements := func(tc []testCase, actual *serverpb.StatementsResponse) {
checkStatements := func(t *testing.T, tc []testCase, actual *serverpb.StatementsResponse) {
t.Helper()
var expectedStatements []string
for _, stmt := range tc {
var expectedStmt = stmt.stmt
Expand Down Expand Up @@ -162,38 +163,44 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
require.Equal(t, expectedStatements, actualStatements)
}

// First we verify that we have expected stats from tenants
checkStatements(testCaseTenant, tenantStats)
checkStatements(testCaseTenant, tenantCombinedStats)
// First we verify that we have expected stats from tenants.
t.Run("tenant-stats", func(t *testing.T) {
checkStatements(t, testCaseTenant, tenantStats)
checkStatements(t, testCaseTenant, tenantCombinedStats)
})

// Now we verify the non tenant stats are what we expected.
checkStatements(testCaseNonTenant, &nonTenantStats)
checkStatements(testCaseNonTenant, &nonTenantCombinedStats)
t.Run("non-tenant-stats", func(t *testing.T) {
checkStatements(t, testCaseNonTenant, &nonTenantStats)
checkStatements(t, testCaseNonTenant, &nonTenantCombinedStats)
})

// Now we verify that tenant and non-tenant have no visibility into each other's stats.
for _, tenantStmt := range tenantStats.Statements {
for _, nonTenantStmt := range nonTenantStats.Statements {
require.NotEqual(t, tenantStmt, nonTenantStmt, "expected tenant to have no visibility to non-tenant's statement stats, but found:", nonTenantStmt)
t.Run("overlap", func(t *testing.T) {
for _, tenantStmt := range tenantStats.Statements {
for _, nonTenantStmt := range nonTenantStats.Statements {
require.NotEqual(t, tenantStmt, nonTenantStmt, "expected tenant to have no visibility to non-tenant's statement stats, but found:", nonTenantStmt)
}
}
}

for _, tenantTxn := range tenantStats.Transactions {
for _, nonTenantTxn := range nonTenantStats.Transactions {
require.NotEqual(t, tenantTxn, nonTenantTxn, "expected tenant to have no visibility to non-tenant's transaction stats, but found:", nonTenantTxn)
for _, tenantTxn := range tenantStats.Transactions {
for _, nonTenantTxn := range nonTenantStats.Transactions {
require.NotEqual(t, tenantTxn, nonTenantTxn, "expected tenant to have no visibility to non-tenant's transaction stats, but found:", nonTenantTxn)
}
}
}

for _, tenantStmt := range tenantCombinedStats.Statements {
for _, nonTenantStmt := range nonTenantCombinedStats.Statements {
require.NotEqual(t, tenantStmt, nonTenantStmt, "expected tenant to have no visibility to non-tenant's statement stats, but found:", nonTenantStmt)
for _, tenantStmt := range tenantCombinedStats.Statements {
for _, nonTenantStmt := range nonTenantCombinedStats.Statements {
require.NotEqual(t, tenantStmt, nonTenantStmt, "expected tenant to have no visibility to non-tenant's statement stats, but found:", nonTenantStmt)
}
}
}

for _, tenantTxn := range tenantCombinedStats.Transactions {
for _, nonTenantTxn := range nonTenantCombinedStats.Transactions {
require.NotEqual(t, tenantTxn, nonTenantTxn, "expected tenant to have no visibility to non-tenant's transaction stats, but found:", nonTenantTxn)
for _, tenantTxn := range tenantCombinedStats.Transactions {
for _, nonTenantTxn := range nonTenantCombinedStats.Transactions {
require.NotEqual(t, tenantTxn, nonTenantTxn, "expected tenant to have no visibility to non-tenant's transaction stats, but found:", nonTenantTxn)
}
}
}
})
}

func TestResetSQLStatsRPCForTenant(t *testing.T) {
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/cert.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,10 +290,10 @@ func runListCerts(cmd *cobra.Command, args []string) error {
var certCmds = []*cobra.Command{
createCACertCmd,
createClientCACertCmd,
mtCreateTenantClientCACertCmd,
mtCreateTenantCACertCmd,
createNodeCertCmd,
createClientCertCmd,
mtCreateTenantClientCertCmd,
mtCreateTenantCertCmd,
listCertsCmd,
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/mt.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ func init() {
mtCmd.AddCommand(mtTestDirectorySvr)

mtCertsCmd.AddCommand(
mtCreateTenantClientCACertCmd,
mtCreateTenantClientCertCmd,
mtCreateTenantCACertCmd,
mtCreateTenantCertCmd,
)

mtCmd.AddCommand(mtCertsCmd)
Expand Down
23 changes: 13 additions & 10 deletions pkg/cli/mt_cert.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@ import (
"github.com/spf13/cobra"
)

// mtCreateTenantClientCACertCmd generates a tenant CA certificate and stores it
// mtCreateTenantCACertCmd generates a tenant CA certificate and stores it
// in the cert directory.
var mtCreateTenantClientCACertCmd = &cobra.Command{
var mtCreateTenantCACertCmd = &cobra.Command{
Use: "create-tenant-client-ca --certs-dir=<path to cockroach certs dir> --ca-key=<path>",
Short: "create tenant client CA certificate and key",
Long: `
Expand All @@ -34,7 +34,7 @@ If the CA certificate exists and --overwrite is true, the new CA certificate is
Args: cobra.NoArgs,
RunE: clierrorplus.MaybeDecorateError(func(cmd *cobra.Command, args []string) error {
return errors.Wrap(
security.CreateTenantClientCAPair(
security.CreateTenantCAPair(
certCtx.certsDir,
certCtx.caKey,
certCtx.keySize,
Expand All @@ -47,8 +47,8 @@ If the CA certificate exists and --overwrite is true, the new CA certificate is

// A createClientCert command generates a client certificate and stores it
// in the cert directory under <username>.crt and key under <username>.key.
var mtCreateTenantClientCertCmd = &cobra.Command{
Use: "create-tenant-client --certs-dir=<path to cockroach certs dir> --ca-key=<path-to-ca-key> <tenant-id>",
var mtCreateTenantCertCmd = &cobra.Command{
Use: "create-tenant-client --certs-dir=<path to cockroach certs dir> --ca-key=<path-to-ca-key> <tenant-id> <host 1> <host 2> ... <host N>",
Short: "create tenant client certificate and key",
Long: `
Generate a tenant client certificate "<certs-dir>/client-tenant.<tenant-id>.crt" and key
Expand All @@ -60,27 +60,30 @@ Requires a CA cert in "<certs-dir>/ca-client-tenant.crt" and matching key in "--
If "ca-client-tenant.crt" contains more than one certificate, the first is used.
Creation fails if the CA expiration time is before the desired certificate expiration.
`,
Args: cobra.ExactArgs(1),
Args: cobra.MinimumNArgs(2),
RunE: clierrorplus.MaybeDecorateError(
func(cmd *cobra.Command, args []string) error {
tenantID, err := strconv.ParseUint(args[0], 10, 64)
tenantIDs := args[0]
hostAddrs := args[1:]
tenantID, err := strconv.ParseUint(tenantIDs, 10, 64)
if err != nil {
return errors.Wrapf(err, "%s is invalid uint64", args[0])
return errors.Wrapf(err, "%s is invalid uint64", tenantIDs)
}
cp, err := security.CreateTenantClientPair(
cp, err := security.CreateTenantPair(
certCtx.certsDir,
certCtx.caKey,
certCtx.keySize,
certCtx.certificateLifetime,
tenantID,
hostAddrs,
)
if err != nil {
return errors.Wrap(
err,
"failed to generate tenant client certificate and key")
}
return errors.Wrap(
security.WriteTenantClientPair(certCtx.certsDir, cp, certCtx.overwriteFiles),
security.WriteTenantPair(certCtx.certsDir, cp, certCtx.overwriteFiles),
"failed to write tenant client certificate and key")
}),
}
37 changes: 32 additions & 5 deletions pkg/rpc/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,13 +121,39 @@ func (a kvAuth) authenticate(ctx context.Context) (roachpb.TenantID, error) {
return roachpb.TenantID{}, err
}

subj := tlsInfo.State.PeerCertificates[0].Subject
if security.Contains(subj.OrganizationalUnit, security.TenantsOU) {
// Tenant authentication.
return tenantFromCommonName(subj.CommonName)
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)
}
} else {
// This node is a SQL tenant server.
//
// Is this a connection from another SQL tenant server?
if security.IsTenantCertificate(clientCert) {
// Incoming connection originating from a tenant SQL server,
// into a KV node. Let through. The other server
// is able to use any of this server's RPCs.
return roachpb.TenantID{}, nil
}
}

// KV auth.
// 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.
//
// In both cases, we must check that the client cert is either root
// or node.

// TODO(benesch): the vast majority of RPCs should be limited to just
// NodeUser. This is not a security concern, as RootUser has access to
Expand All @@ -137,5 +163,6 @@ func (a kvAuth) authenticate(ctx context.Context) (roachpb.TenantID, error) {
!security.Contains(certUsers, security.RootUser) {
return roachpb.TenantID{}, authErrorf("user %s is not allowed to perform this RPC", certUsers)
}

return roachpb.TenantID{}, nil
}
2 changes: 2 additions & 0 deletions pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@ import (
// server, that is, it ensures that the request only accesses resources
// available to the tenant.
type tenantAuthorizer struct {
// tenantID is the tenant ID for the current node.
// Equals SystemTenantID when running a KV node.
tenantID roachpb.TenantID
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/rpc/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func TestTenantFromCert(t *testing.T) {
p := peer.Peer{AuthInfo: tlsInfo}
ctx := peer.NewContext(context.Background(), &p)

tenID, err := kvAuth{}.authenticate(ctx)
tenID, err := kvAuth{tenant: tenantAuthorizer{tenantID: roachpb.SystemTenantID}}.authenticate(ctx)

if tc.expErr == "" {
require.Equal(t, tc.expTenID, tenID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/rpc/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -729,7 +729,7 @@ func (ctx *Context) grpcDialOptions(
if ctx.tenID == roachpb.SystemTenantID {
tlsConfig, err = ctx.GetClientTLSConfig()
} else {
tlsConfig, err = ctx.GetTenantClientTLSConfig()
tlsConfig, err = ctx.GetTenantTLSConfig()
}
if err != nil {
return nil, err
Expand Down
9 changes: 6 additions & 3 deletions pkg/rpc/tls.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,9 @@ type SecurityContext struct {
func MakeSecurityContext(
cfg *base.Config, tlsSettings security.TLSSettings, tenID roachpb.TenantID,
) SecurityContext {
if tenID.ToUint64() == 0 {
panic(errors.AssertionFailedf("programming error: tenant ID not defined"))
}
return SecurityContext{
CertsLocator: security.MakeCertsLocator(cfg.SSLCertsDir),
TLSSettings: tlsSettings,
Expand Down Expand Up @@ -151,13 +154,13 @@ func (ctx *SecurityContext) GetClientTLSConfig() (*tls.Config, error) {
return tlsCfg, nil
}

// GetTenantClientTLSConfig returns the client TLS config for the tenant, provided
// GetTenantTLSConfig returns the client TLS config for the tenant, provided
// the SecurityContext operates on behalf of a secondary tenant (i.e. not the
// system tenant).
//
// If Insecure is true, return a nil config, otherwise retrieves the client
// certificate for the configured tenant from the cert manager.
func (ctx *SecurityContext) GetTenantClientTLSConfig() (*tls.Config, error) {
func (ctx *SecurityContext) GetTenantTLSConfig() (*tls.Config, error) {
// Early out.
if ctx.config.Insecure {
return nil, nil
Expand All @@ -168,7 +171,7 @@ func (ctx *SecurityContext) GetTenantClientTLSConfig() (*tls.Config, error) {
return nil, wrapError(err)
}

tlsCfg, err := cm.GetTenantClientTLSConfig()
tlsCfg, err := cm.GetTenantTLSConfig()
if err != nil {
return nil, wrapError(err)
}
Expand Down
9 changes: 7 additions & 2 deletions pkg/security/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,8 +128,7 @@ func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAut
// The client certificate should not be a tenant client type. For now just
// check that it doesn't have OU=Tenants. It would make sense to add
// explicit OU=Users to all client certificates and to check for match.
ous := tlsState.PeerCertificates[0].Subject.OrganizationalUnit
if Contains(ous, TenantsOU) {
if IsTenantCertificate(tlsState.PeerCertificates[0]) {
return nil,
errors.Errorf("using tenant client certificate as user certificate is not allowed")
}
Expand All @@ -143,6 +142,12 @@ func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAut
}, nil
}

// IsTenantCertificate returns true if the passed certificate indicates an
// inbound Tenant connection.
func IsTenantCertificate(cert *x509.Certificate) bool {
return Contains(cert.Subject.OrganizationalUnit, TenantsOU)
}

// UserAuthPasswordHook builds an authentication hook based on the security
// mode, password, and its potentially matching hash.
func UserAuthPasswordHook(insecureMode bool, password string, hashedPassword []byte) UserAuthHook {
Expand Down
Loading

0 comments on commit 406b396

Please sign in to comment.