Skip to content

Commit

Permalink
Merge pull request #111006 from rafiss/backport23.1-110726
Browse files Browse the repository at this point in the history
release-23.1: security: clear client cert expiration cache on SIGHUP
  • Loading branch information
rafiss authored Sep 21, 2023
2 parents 4a98200 + 277f8cc commit 428b331
Show file tree
Hide file tree
Showing 10 changed files with 99 additions and 21 deletions.
13 changes: 13 additions & 0 deletions pkg/security/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -106,54 +106,67 @@ go_test(
] + select({
"@io_bazel_rules_go//go/platform:aix": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:android": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:darwin": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:dragonfly": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:freebsd": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:illumos": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:ios": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:js": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:linux": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:netbsd": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:openbsd": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:plan9": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"@io_bazel_rules_go//go/platform:solaris": [
"//pkg/util/log/eventpb",
"@com_github_prometheus_client_model//go",
"@org_golang_x_sys//unix",
],
"//conditions:default": [],
Expand Down
6 changes: 2 additions & 4 deletions pkg/security/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,6 @@ func UserAuthCertHook(
tlsState *tls.ConnectionState,
tenantID roachpb.TenantID,
certManager *CertificateManager,
cache *ClientCertExpirationCache,
) (UserAuthHook, error) {
var certUserScope []CertificateUserScope
if !insecureMode {
Expand Down Expand Up @@ -193,11 +192,10 @@ func UserAuthCertHook(

if ValidateUserScope(certUserScope, systemIdentity.Normalized(), tenantID) {
if certManager != nil {
cache.MaybeUpsert(
certManager.MaybeUpsertClientExpiration(
ctx,
systemIdentity.Normalized(),
systemIdentity,
peerCert.NotAfter.Unix(),
certManager.certMetrics.ClientExpiration,
)
}
return nil
Expand Down
1 change: 0 additions & 1 deletion pkg/security/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,6 @@ func TestAuthenticationHook(t *testing.T) {
makeFakeTLSState(t, tc.tlsSpec),
tc.tenantID,
nil, /* certManager */
nil, /* cache */
)
if (err == nil) != tc.buildHookSuccess {
t.Fatalf("expected success=%t, got err=%v", tc.buildHookSuccess, err)
Expand Down
3 changes: 3 additions & 0 deletions pkg/security/cert_expiry_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,9 @@ func NewClientCertExpirationCache(
},
OnEvictedEntry: func(entry *cache.Entry) {
gauge := entry.Value.(*aggmetric.Gauge)
// The child metric will continue to report into the parent metric even
// after unlinking, so we also reset it to 0.
gauge.Update(0)
gauge.Unlink()
c.mu.acc.Shrink(ctx, int64(unsafe.Sizeof(*gauge)))
},
Expand Down
27 changes: 27 additions & 0 deletions pkg/security/certificate_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,9 @@ type CertificateManager struct {
// own locking.
certMetrics Metrics

// Client cert expiration cache.
clientCertExpirationCache *ClientCertExpirationCache

// mu protects all remaining fields.
mu syncutil.RWMutex

Expand Down Expand Up @@ -167,6 +170,9 @@ func (cm *CertificateManager) RegisterSignalHandler(
return
case sig := <-ch:
log.Ops.Infof(ctx, "received signal %q, triggering certificate reload", sig)
if cache := cm.clientCertExpirationCache; cache != nil {
cache.Clear()
}
if err := cm.LoadCertificates(); err != nil {
log.Ops.Warningf(ctx, "could not reload certificates: %v", err)
log.StructuredEvent(ctx, &eventpb.CertsReload{Success: false, ErrorMessage: err.Error()})
Expand All @@ -178,6 +184,27 @@ func (cm *CertificateManager) RegisterSignalHandler(
})
}

// RegisterExpirationCache registers a cache for client certificate expiration.
// It is called during server startup.
func (cm *CertificateManager) RegisterExpirationCache(cache *ClientCertExpirationCache) {
cm.clientCertExpirationCache = cache
}

// MaybeUpsertClientExpiration updates or inserts the expiration time for the
// given client certificate. An update is contingent on whether the old
// expiration is after the new expiration.
func (cm *CertificateManager) MaybeUpsertClientExpiration(
ctx context.Context, identity username.SQLUsername, expiration int64,
) {
if cache := cm.clientCertExpirationCache; cache != nil {
cache.MaybeUpsert(ctx,
identity.Normalized(),
expiration,
cm.certMetrics.ClientExpiration,
)
}
}

// CACert returns the CA cert. May be nil.
// Callers should check for an internal Error field.
func (cm *CertificateManager) CACert() *CertInfo {
Expand Down
42 changes: 38 additions & 4 deletions pkg/security/certs_rotation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"regexp"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer"
Expand All @@ -41,6 +42,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
prometheusgo "github.com/prometheus/client_model/go"
"github.com/stretchr/testify/require"
"golang.org/x/sys/unix"
)

Expand All @@ -56,7 +59,7 @@ func TestRotateCerts(t *testing.T) {
defer ResetTest()
certsDir := t.TempDir()

if err := generateBaseCerts(certsDir); err != nil {
if err := generateBaseCerts(certsDir, 48*time.Hour); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -99,6 +102,22 @@ func TestRotateCerts(t *testing.T) {
return goDB
}

checkCertExpirationMetric := func() int64 {
cm, err := s.RPCContext().SecurityContext.GetCertificateManager()
if err != nil {
t.Fatal(err)
}
ret := int64(0)
cm.Metrics().ClientExpiration.Each(nil, func(pm *prometheusgo.Metric) {
for _, l := range pm.GetLabel() {
if l.GetName() == "sql_user" && l.GetValue() == username.RootUser {
ret = int64(pm.GetGauge().GetValue())
}
}
})
return ret
}

// Some errors codes.
const kBadAuthority = "certificate signed by unknown authority"
const kBadCertificate = "tls: bad certificate"
Expand All @@ -117,6 +136,9 @@ func TestRotateCerts(t *testing.T) {
t.Fatalf("could not create http client: %v", err)
}

// Before any client has connected, the expiration metric should be zero.
require.Zero(t, checkCertExpirationMetric())

if err := clientTest(firstClient); err != nil {
t.Fatal(err)
}
Expand All @@ -128,12 +150,15 @@ func TestRotateCerts(t *testing.T) {
t.Fatal(err)
}

// Delete certs and re-generate them.
// Now the expiration metric should be set.
require.Greater(t, checkCertExpirationMetric(), timeutil.Now().Add(40*time.Hour).Unix())

// Delete certs and re-generate them with a longer client cert lifetime.
// New clients will fail with CA errors.
if err := os.RemoveAll(certsDir); err != nil {
t.Fatal(err)
}
if err := generateBaseCerts(certsDir); err != nil {
if err := generateBaseCerts(certsDir, 54*time.Hour); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -195,6 +220,9 @@ func TestRotateCerts(t *testing.T) {
return nil
})

// The expiration metric should be zero after the SIGHUP.
require.Zero(t, checkCertExpirationMetric())

// Check that the structured event was logged.
// We use SucceedsSoon here because there may be a delay between
// the moment SIGHUP is processed and certs are reloaded, and
Expand Down Expand Up @@ -242,13 +270,16 @@ func TestRotateCerts(t *testing.T) {
t.Fatal(err)
}

// The expiration metric should be set again, but should now show a higher value.
require.Greater(t, checkCertExpirationMetric(), timeutil.Now().Add(50*time.Hour).Unix())

// Now regenerate certs, but keep the CA cert around.
// We still need to delete the key.
// New clients with certs will fail with bad certificate (CA not yet loaded).
if err := os.Remove(filepath.Join(certsDir, certnames.EmbeddedCAKey)); err != nil {
t.Fatal(err)
}
if err := generateBaseCerts(certsDir); err != nil {
if err := generateBaseCerts(certsDir, 58*time.Hour); err != nil {
t.Fatal(err)
}

Expand Down Expand Up @@ -301,6 +332,9 @@ func TestRotateCerts(t *testing.T) {
}
return nil
})

// The expiration metric should be updated to be larger again.
require.Greater(t, checkCertExpirationMetric(), timeutil.Now().Add(57*time.Hour).Unix())
}

var cmLogRe = regexp.MustCompile(`event_log\.go`)
8 changes: 4 additions & 4 deletions pkg/security/certs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -263,7 +263,7 @@ func TestGenerateNodeCerts(t *testing.T) {
// client.root.crt: client certificate for the root user.
// client-tenant.10.crt: tenant client certificate for tenant 10.
// tenant-signing.10.crt: tenant signing certificate for tenant 10.
func generateBaseCerts(certsDir string) error {
func generateBaseCerts(certsDir string, clientCertLifetime time.Duration) error {
{
caKey := filepath.Join(certsDir, certnames.EmbeddedCAKey)

Expand All @@ -285,7 +285,7 @@ func generateBaseCerts(certsDir string) error {
certsDir,
caKey,
testKeySize,
time.Hour*48,
clientCertLifetime,
true,
username.RootUserName(),
[]roachpb.TenantID{roachpb.SystemTenantID},
Expand Down Expand Up @@ -328,7 +328,7 @@ func generateBaseCerts(certsDir string) error {
// client.node.crt: node client cert: signed by ca-client.crt
// client.root.crt: root client cert: signed by ca-client.crt
func generateSplitCACerts(certsDir string) error {
if err := generateBaseCerts(certsDir); err != nil {
if err := generateBaseCerts(certsDir, 48*time.Hour); err != nil {
return err
}

Expand Down Expand Up @@ -381,7 +381,7 @@ func TestUseCerts(t *testing.T) {
defer ResetTest()
certsDir := t.TempDir()

if err := generateBaseCerts(certsDir); err != nil {
if err := generateBaseCerts(certsDir, 48*time.Hour); err != nil {
t.Fatal(err)
}

Expand Down
15 changes: 12 additions & 3 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -944,6 +944,18 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
&contentionMetrics,
)

if !cfg.Insecure {
certMgr, err := cfg.rpcContext.SecurityContext.GetCertificateManager()
if err != nil {
return nil, errors.Wrap(err, "initializing certificate manager")
}
certMgr.RegisterExpirationCache(
security.NewClientCertExpirationCache(
ctx, cfg.Settings, cfg.stopper, &timeutil.DefaultTimeSource{}, rootSQLMemoryMonitor,
),
)
}

storageEngineClient := kvserver.NewStorageEngineClient(cfg.nodeDialer)
*execCfg = sql.ExecutorConfig{
Settings: cfg.Settings,
Expand Down Expand Up @@ -982,9 +994,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
AuditConfig: &auditlogging.AuditConfigLock{
Config: auditlogging.EmptyAuditConfig(),
},
ClientCertExpirationCache: security.NewClientCertExpirationCache(
ctx, cfg.Settings, cfg.stopper, &timeutil.DefaultTimeSource{}, rootSQLMemoryMonitor,
),
RootMemoryMonitor: rootSQLMemoryMonitor,
TestingKnobs: sqlExecutorTestingKnobs,
CompactEngineSpanFunc: storageEngineClient.CompactEngineSpan,
Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/obs"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server/autoconfig/acprovider"
"github.com/cockroachdb/cockroach/pkg/server/pgurl"
Expand Down Expand Up @@ -1313,9 +1312,6 @@ type ExecutorConfig struct {
// Role membership cache.
RoleMemberCache *MembershipCache

// Client cert expiration cache.
ClientCertExpirationCache *security.ClientCertExpirationCache

// SessionInitCache cache; contains information used during authentication
// and per-role default settings.
SessionInitCache *sessioninit.Cache
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/pgwire/auth_methods.go
Original file line number Diff line number Diff line change
Expand Up @@ -442,7 +442,6 @@ func authCert(
&tlsState,
execCfg.RPCContext.TenantID,
cm,
execCfg.ClientCertExpirationCache,
)
if err != nil {
return err
Expand Down

0 comments on commit 428b331

Please sign in to comment.