Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
110726: security: clear client cert expiration cache on SIGHUP r=rafiss a=rafiss

Fixes #110493.
Epic: CRDB-28893

Release note (security update): The SIGHUP signal will now clear the
cached expiration times for client certificates that are reported by
the security.certificate.expiration.client metric. (The SIGHUP signal also
will still cause the server to reload certificates from the filesystem, as
it did before.)

110977: kv: mark kvnemesis tests as "medium" sized r=nvanbenschoten a=nvanbenschoten

Fixes #110464.
Fixes #110519.

This commit marks the kvnemesis tests as "medium" size and increases their timeout from 55s to 295s, which is inline with other "medium" size tests. The slowest of the tests in this package is `TestKVNemesisMultiNode`, which regularly takes about 35s to run on a gceworker. It's not surprising that we were occasionally seeing timeouts in CI with the 55s timeout.

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
3 people committed Sep 20, 2023
3 parents 8bf5ade + 8c42665 + 93cc706 commit d2cf1f8
Show file tree
Hide file tree
Showing 11 changed files with 101 additions and 23 deletions.
4 changes: 2 additions & 2 deletions pkg/kv/kvnemesis/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ go_library(

go_test(
name = "kvnemesis_test",
size = "small",
size = "medium",
srcs = [
"applier_test.go",
"engine_test.go",
Expand All @@ -68,7 +68,7 @@ go_test(
"operations_test.go",
"validator_test.go",
],
args = ["-test.timeout=55s"],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
embed = [":kvnemesis"],
deps = [
Expand Down
13 changes: 13 additions & 0 deletions pkg/security/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -105,54 +105,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 @@ -103,6 +106,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 @@ -120,6 +139,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 @@ -131,12 +153,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 @@ -197,6 +222,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 @@ -244,13 +272,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 @@ -302,6 +333,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 @@ -264,7 +264,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 @@ -286,7 +286,7 @@ func generateBaseCerts(certsDir string) error {
certsDir,
caKey,
testKeySize,
time.Hour*48,
clientCertLifetime,
true,
username.RootUserName(),
[]roachpb.TenantID{roachpb.SystemTenantID},
Expand Down Expand Up @@ -329,7 +329,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 @@ -384,7 +384,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 @@ -922,6 +922,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.kvNodeDialer)
*execCfg = sql.ExecutorConfig{
Settings: cfg.Settings,
Expand Down Expand Up @@ -960,9 +972,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 @@ -50,7 +50,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 @@ -1302,9 +1301,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 d2cf1f8

Please sign in to comment.