Skip to content

Commit

Permalink
Merge #103592
Browse files Browse the repository at this point in the history
103592: security: add a cache to store client cert expirations for users r=sean-,knz a=cameronnunez

Fixes #100699.

This change introduces a new node-level, FIFO cache for storing the minimum time-until-expiration of the set of client certs seen (per user). 

Entries in the cache are username-gauge pairs, where the gauges contain the expiration values. Periodically, gauges with 
expiration values that are not in the future will be cleared from the cache. The purging process triggers hourly. 

The gauges in the cache are children of an aggregate gauge `ClientExpiration`, so the aggregated value for that metric 
is not meaningful and only serves as a link to the children. `ClientExpiration` (`security.certificate.expiration.client`)
is exported as a metric and the children gauges are labeled by SQL user.

Release note (security update): There is a new `server.client_cert_expiration_cache.capacity` setting which, when set to a non-zero number, makes it so that the minimum time-until-expiration of the set of client certificates seen is stored (for every user). This setting can be used to ensure client cert expirations are exported as a metric (if set to zero, the metric `security.certificate.expiration.client` will have a value of zero).

Co-authored-by: Cameron Nunez <[email protected]>
  • Loading branch information
craig[bot] and cameronnunez committed May 31, 2023
2 parents 3c625c6 + abdaefa commit 906e7ad
Show file tree
Hide file tree
Showing 12 changed files with 674 additions and 111 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ server.auth_log.sql_connections.enabled boolean false if set, log SQL client con
server.auth_log.sql_sessions.enabled boolean false if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes) tenant-rw
server.authentication_cache.enabled boolean true enables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related information tenant-rw
server.child_metrics.enabled boolean false enables the exporting of child metrics, additional prometheus time series with extra labels tenant-rw
server.client_cert_expiration_cache.capacity integer 1000 the maximum number of client cert expirations stored tenant-rw
server.clock.forward_jump_check_enabled boolean false if enabled, forward clock jumps > max_offset/2 will cause a panic tenant-rw
server.clock.persist_upper_bound_interval duration 0s the interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature. tenant-rw
server.eventlog.enabled boolean true if set, logged notable events are also stored in the table system.eventlog tenant-rw
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@
<tr><td><div id="setting-server-auth-log-sql-sessions-enabled" class="anchored"><code>server.auth_log.sql_sessions.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes)</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-authentication-cache-enabled" class="anchored"><code>server.authentication_cache.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>enables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related information</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-child-metrics-enabled" class="anchored"><code>server.child_metrics.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>enables the exporting of child metrics, additional prometheus time series with extra labels</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-client-cert-expiration-cache-capacity" class="anchored"><code>server.client_cert_expiration_cache.capacity</code></div></td><td>integer</td><td><code>1000</code></td><td>the maximum number of client cert expirations stored</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-clock-forward-jump-check-enabled" class="anchored"><code>server.clock.forward_jump_check_enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if enabled, forward clock jumps &gt; max_offset/2 will cause a panic</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-clock-persist-upper-bound-interval" class="anchored"><code>server.clock.persist_upper_bound_interval</code></div></td><td>duration</td><td><code>0s</code></td><td>the interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-server-consistency-check-max-rate" class="anchored"><code>server.consistency_check.max_rate</code></div></td><td>byte size</td><td><code>8.0 MiB</code></td><td>the rate limit (bytes/sec) to use for consistency checks; used in conjunction with server.consistency_check.interval to control the frequency of consistency checks. Note that setting this too high can negatively impact performance.</td><td>Dedicated/Self-Hosted</td></tr>
Expand Down
11 changes: 11 additions & 0 deletions pkg/security/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,10 @@ go_library(
srcs = [
"auth.go",
"auto_tls_init.go",
"cert_expiry_cache.go",
"certificate_loader.go",
"certificate_manager.go",
"certificate_metrics.go",
"certs.go",
"join_token.go",
"ocsp.go",
Expand All @@ -31,11 +33,14 @@ go_library(
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/util/cache",
"//pkg/util/encoding",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/metric",
"//pkg/util/metric/aggmetric",
"//pkg/util/mon",
"//pkg/util/quotapool",
"//pkg/util/randutil",
"//pkg/util/stop",
Expand All @@ -58,6 +63,7 @@ go_test(
srcs = [
"auth_test.go",
"auto_tls_init_test.go",
"cert_expiry_cache_test.go",
"certificate_loader_test.go",
"certificate_manager_test.go",
"certs_rotation_test.go",
Expand All @@ -81,12 +87,17 @@ go_test(
"//pkg/security/securitytest",
"//pkg/security/username",
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/util/envutil",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/metric/aggmetric",
"//pkg/util/mon",
"//pkg/util/randutil",
"//pkg/util/stop",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
Expand Down
18 changes: 16 additions & 2 deletions pkg/security/auth.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,11 @@ 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, tenantID roachpb.TenantID,
insecureMode bool,
tlsState *tls.ConnectionState,
tenantID roachpb.TenantID,
certManager *CertificateManager,
cache *ClientCertExpirationCache,
) (UserAuthHook, error) {
var certUserScope []CertificateUserScope
if !insecureMode {
Expand Down Expand Up @@ -177,14 +181,24 @@ func UserAuthCertHook(
return nil
}

peerCert := tlsState.PeerCertificates[0]

// 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.
if IsTenantCertificate(tlsState.PeerCertificates[0]) {
if IsTenantCertificate(peerCert) {
return errors.Errorf("using tenant client certificate as user certificate is not allowed")
}

if ValidateUserScope(certUserScope, systemIdentity.Normalized(), tenantID) {
if certManager != nil {
cache.MaybeUpsert(
ctx,
systemIdentity.Normalized(),
peerCert.NotAfter.Unix(),
certManager.certMetrics.ClientExpiration,
)
}
return nil
}
return errors.WithDetailf(errors.Errorf("certificate authentication failed for user %q", systemIdentity),
Expand Down
8 changes: 7 additions & 1 deletion pkg/security/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -321,7 +321,13 @@ func TestAuthenticationHook(t *testing.T) {
if err != nil {
t.Fatal(err)
}
hook, err := security.UserAuthCertHook(tc.insecure, makeFakeTLSState(t, tc.tlsSpec), tc.tenantID)
hook, err := security.UserAuthCertHook(
tc.insecure,
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
230 changes: 230 additions & 0 deletions pkg/security/cert_expiry_cache.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,230 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package security

import (
"context"
math_rand "math/rand"
"time"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/cache"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric/aggmetric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)

// CacheCapacityMax is set arbitrarily high; configurable later if needed.
const CacheCapacityMax = 65000

// ClientCertExpirationCacheCapacity is the cluster setting that controls the
// maximum number of client cert expirations in the cache.
var ClientCertExpirationCacheCapacity = settings.RegisterIntSetting(
settings.TenantWritable,
"server.client_cert_expiration_cache.capacity",
"the maximum number of client cert expirations stored",
1000,
).WithPublic()

// ClientCertExpirationCache contains a cache of gauge objects keyed by
// SQL username strings. It is a FIFO cache that stores gauges valued by
// minimum expiration of the client certs seen (per user).
type ClientCertExpirationCache struct {
mu struct {
// NB: Cannot be a RWMutex for Get because UnorderedCache.Get manipulates
// an internal hashmap.
syncutil.Mutex
cache *cache.UnorderedCache
acc mon.BoundAccount
}
settings *cluster.Settings
stopper *stop.Stopper
mon *mon.BytesMonitor
timeSrc interface{}
}

// NewClientCertExpirationCache creates a new client cert expiration cache.
func NewClientCertExpirationCache(
ctx context.Context,
st *cluster.Settings,
stopper *stop.Stopper,
timeSrc timeutil.TimeSource,
parentMon *mon.BytesMonitor,
) *ClientCertExpirationCache {
c := &ClientCertExpirationCache{settings: st}
c.stopper = stopper

switch timeSrc := timeSrc.(type) {
case *timeutil.DefaultTimeSource, *timeutil.ManualTime:
c.timeSrc = timeSrc
default:
c.timeSrc = &timeutil.DefaultTimeSource{}
}

c.mu.cache = cache.NewUnorderedCache(cache.Config{
Policy: cache.CacheFIFO,
ShouldEvict: func(size int, _, value interface{}) bool {
var capacity int64
settingCapacity := ClientCertExpirationCacheCapacity.Get(&st.SV)
if settingCapacity < CacheCapacityMax {
capacity = settingCapacity
} else {
capacity = CacheCapacityMax
}
return int64(size) > capacity
},
OnEvictedEntry: func(entry *cache.Entry) {
gauge := entry.Value.(*aggmetric.Gauge)
gauge.Unlink()
c.mu.acc.Shrink(ctx, int64(unsafe.Sizeof(*gauge)))
},
})
c.mon = mon.NewMonitorInheritWithLimit(
"client-expiration-cache", 0 /* limit */, parentMon,
)
c.mu.acc = c.mon.MakeBoundAccount()
c.mon.StartNoReserved(ctx, parentMon)

// Begin an async task to periodically evict entries associated with
// expiration values that are in the past.
if err := c.startPurgePastExpirations(ctx); err != nil {
log.Ops.Warningf(
ctx, "failed to initiate periodic purge of expiration cache entries: %v", err,
)
}

return c
}

// Get retrieves the cert expiration for the given username, if it exists.
// An expiration of 0 indicates an entry was not found.
func (c *ClientCertExpirationCache) Get(key string) (int64, bool) {
c.mu.Lock()
defer c.mu.Unlock()
value, ok := c.mu.cache.Get(key)
if !ok {
return 0, ok
}
// If the expiration has already been reached, remove the entry and indicate
// that the entry was not found.
gauge := value.(*aggmetric.Gauge)
if gauge.Value() < c.timeNow() {
c.mu.cache.Del(key)
return 0, false
}
return gauge.Value(), ok
}

// MaybeUpsert may update or insert a client cert expiration gauge for a
// particular user into the cache. An update is contingent on whether the
// old expiration is after the new expiration. This ensures that the cache
// maintains the minimum expiration for each user.
func (c *ClientCertExpirationCache) MaybeUpsert(
ctx context.Context, key string, newExpiry int64, parentGauge *aggmetric.AggGauge,
) {
c.mu.Lock()
defer c.mu.Unlock()

value, ok := c.mu.cache.Get(key)
if !ok {
err := c.mu.acc.Grow(ctx, int64(unsafe.Sizeof(aggmetric.Gauge{})))
if err == nil {
// Only create new gauges for expirations in the future.
if newExpiry > c.timeNow() {
gauge := parentGauge.AddChild(key)
gauge.Update(newExpiry)
c.mu.cache.Add(key, gauge)
}
} else {
log.Ops.Warningf(ctx, "no memory available to cache cert expiry: %v", err)
}
} else if gauge := value.(*aggmetric.Gauge); newExpiry < gauge.Value() || gauge.Value() == 0 {
gauge.Update(newExpiry)
}
}

// Clear removes all entries from the cache.
func (c *ClientCertExpirationCache) Clear() {
c.mu.Lock()
defer c.mu.Unlock()
c.mu.cache.Clear()
}

// Len returns the number of cert expirations in the cache.
func (c *ClientCertExpirationCache) Len() int {
c.mu.Lock()
defer c.mu.Unlock()
return c.mu.cache.Len()
}

// timeNow returns the current time depending on the time source of the cache.
func (c *ClientCertExpirationCache) timeNow() int64 {
if timeSrc, ok := c.timeSrc.(timeutil.TimeSource); ok {
return timeSrc.Now().Unix()
}
return timeutil.Now().Unix()
}

// startPurgePastExpirations runs an infinite loop in a goroutine which
// regularly evicts entries associated with expiration values that have already
// passed.
func (c *ClientCertExpirationCache) startPurgePastExpirations(ctx context.Context) error {
return c.stopper.RunAsyncTask(ctx, "purge-cert-expiry-cache", func(context.Context) {
const period = time.Hour

timer := timeutil.NewTimer()
defer timer.Stop()

timer.Reset(jitteredInterval(period))
for ; ; timer.Reset(period) {
select {
case <-timer.C:
timer.Read = true
c.PurgePastExpirations()
case <-c.stopper.ShouldQuiesce():
return
case <-ctx.Done():
return
}
}
},
)
}

// PurgePastExpirations removes entries associated with expiration values that
// have already passed. This helps ensure that the cache contains gauges
// with expiration values in the future only.
func (c *ClientCertExpirationCache) PurgePastExpirations() {
c.mu.Lock()
defer c.mu.Unlock()
var deleteEntryKeys []interface{}
now := c.timeNow()
c.mu.cache.Do(func(entry *cache.Entry) {
gauge := entry.Value.(*aggmetric.Gauge)
if gauge.Value() <= now {
deleteEntryKeys = append(deleteEntryKeys, entry.Key)
}
})
for _, key := range deleteEntryKeys {
c.mu.cache.Del(key)
}
}

// jitteredInterval returns a randomly jittered (+/-25%) duration
// from the interval.
func jitteredInterval(interval time.Duration) time.Duration {
return time.Duration(float64(interval) * (0.75 + 0.5*math_rand.Float64()))
}
Loading

0 comments on commit 906e7ad

Please sign in to comment.