Skip to content

Commit

Permalink
Merge #103759
Browse files Browse the repository at this point in the history
103759: sql: add `sql.log.user_audit.reduced_config.enabled` to support single role checks for role-based audit logs r=THardy98 a=THardy98

Part of: #100467

This change introduces the `sql.log.user_audit.reduced_config.enabled`
cluster setting. When enabled, this cluster setting computes a "reduced"
audit configuration based on the user's current role memberships and the
current value for the `sql.log.user_audit` cluster setting. The
"reduced" audit configuration is computed at the _first SQL event emit
by the user after the setting is enabled_. Enabling the cluster setting
allows us to compute the audit configuration once at session start,
instead of computing at each SQL event (providing ~5% increase in
throughput). The tradeoff is that changes to the audit configuration
(user role memberships or cluster setting configuration) are not
reflected within session. Users will need to start a new session to see
these changes in their auditing behaviour.

Release note (sql change): Introduce new cluster setting
`sql.log.user_audit.reduced_config.enabled`. When enabled, this cluster
setting computes a "reduced" audit configuration based on the user's
current role memberships and the current value for the
`sql.log.user_audit` cluster setting. The "reduced" audit configuration
is computed at the _first SQL event emit by the user after the setting
is enabled_. Enabling the cluster setting allows us to compute the audit
configuration once at session start, instead of computing at each SQL
event (providing ~5% increase in throughput). The tradeoff is that
changes to the audit configuration (cluster
setting) and the user's role memberships are not reflected within session. Users will need
to start a new session to see these changes in their auditing behaviour.

Co-authored-by: Thomas Hardy <[email protected]>
  • Loading branch information
craig[bot] and Thomas Hardy committed Jun 6, 2023
2 parents d660d51 + 24597e4 commit 530e2ef
Show file tree
Hide file tree
Showing 9 changed files with 200 additions and 1 deletion.
2 changes: 2 additions & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,8 @@ sql.insights.latency_threshold duration 100ms amount of time after which an exec
sql.log.slow_query.experimental_full_table_scans.enabled boolean false when set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect. tenant-rw
sql.log.slow_query.internal_queries.enabled boolean false when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. tenant-rw
sql.log.slow_query.latency_threshold duration 0s when set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node tenant-rw
sql.log.user_audit string user/role-based audit logging configuration tenant-rw
sql.log.user_audit.reduced_config.enabled boolean false enables logic to compute a reduced audit configuration, computing the audit configuration only once at session start instead of at each SQL event. The tradeoff with the increase in performance (~5%), is that changes to the audit configuration (user role memberships/cluster setting) are not reflected within session. Users will need to start a new session to see these changes in their auditing behaviour. tenant-rw
sql.metrics.index_usage_stats.enabled boolean true collect per index usage statistics tenant-rw
sql.metrics.max_mem_reported_stmt_fingerprints integer 100000 the maximum number of reported statement fingerprints stored in memory tenant-rw
sql.metrics.max_mem_reported_txn_fingerprints integer 100000 the maximum number of reported transaction fingerprints stored in memory tenant-rw
Expand Down
2 changes: 2 additions & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,8 @@
<tr><td><div id="setting-sql-log-slow-query-experimental-full-table-scans-enabled" class="anchored"><code>sql.log.slow_query.experimental_full_table_scans.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>when set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-log-slow-query-internal-queries-enabled" class="anchored"><code>sql.log.slow_query.internal_queries.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-log-slow-query-latency-threshold" class="anchored"><code>sql.log.slow_query.latency_threshold</code></div></td><td>duration</td><td><code>0s</code></td><td>when set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-log-user-audit" class="anchored"><code>sql.log.user_audit</code></div></td><td>string</td><td><code></code></td><td>user/role-based audit logging configuration</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-log-user-audit-reduced-config-enabled" class="anchored"><code>sql.log.user_audit.reduced_config.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>enables logic to compute a reduced audit configuration, computing the audit configuration only once at session start instead of at each SQL event. The tradeoff with the increase in performance (~5%), is that changes to the audit configuration (user role memberships/cluster setting) are not reflected within session. Users will need to start a new session to see these changes in their auditing behaviour.</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-metrics-index-usage-stats-enabled" class="anchored"><code>sql.metrics.index_usage_stats.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>collect per index usage statistics</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-metrics-max-mem-reported-stmt-fingerprints" class="anchored"><code>sql.metrics.max_mem_reported_stmt_fingerprints</code></div></td><td>integer</td><td><code>100000</code></td><td>the maximum number of reported statement fingerprints stored in memory</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-sql-metrics-max-mem-reported-txn-fingerprints" class="anchored"><code>sql.metrics.max_mem_reported_txn_fingerprints</code></div></td><td>integer</td><td><code>100000</code></td><td>the maximum number of reported transaction fingerprints stored in memory</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
Expand Down
1 change: 1 addition & 0 deletions pkg/settings/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,7 @@ var retiredSettings = map[string]struct{}{
// renamed.
"spanconfig.host_coalesce_adjacent.enabled": {},
"sql.defaults.experimental_stream_replication.enabled": {},
"sql.log.unstructured_entries.enabled": {},
}

// sqlDefaultSettings is the list of "grandfathered" existing sql.defaults
Expand Down
46 changes: 46 additions & 0 deletions pkg/sql/audit_logging.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,12 @@ func (p *planner) maybeAuditRoleBasedAuditEvent(ctx context.Context) {
return
}

// Use reduced audit config is enabled.
if auditlogging.UserAuditEnableReducedConfig.Get(&p.execCfg.Settings.SV) {
p.logReducedAuditConfig(ctx)
return
}

user := p.User()
userRoles, err := p.MemberOfWithAdminOption(ctx, user)
if err != nil {
Expand All @@ -68,6 +74,46 @@ func (p *planner) maybeAuditRoleBasedAuditEvent(ctx context.Context) {
}
}

func (p *planner) logReducedAuditConfig(ctx context.Context) {
if !p.reducedAuditConfig.Initialized {
p.initializeReducedAuditConfig(ctx)
}

// Return early if no matching audit setting was found.
if p.reducedAuditConfig.AuditSetting == nil {
return
}

if p.reducedAuditConfig.AuditSetting.IncludeStatements {
p.curPlan.auditEventBuilders = append(p.curPlan.auditEventBuilders,
&auditevents.RoleBasedAuditEvent{
Role: p.reducedAuditConfig.AuditSetting.Role.Normalized(),
},
)
}
}

func (p *planner) initializeReducedAuditConfig(ctx context.Context) {
p.reducedAuditConfig.Lock()
defer p.reducedAuditConfig.Unlock()
if p.reducedAuditConfig.Initialized {
return
}
// Set the initialized flag to true, even for an attempt that errors.
// We do this to avoid the potential overhead of continuously retrying
// to fetch user role memberships.
p.reducedAuditConfig.Initialized = true

user := p.User()
userRoles, err := p.MemberOfWithAdminOption(ctx, user)
if err != nil {
log.Errorf(ctx, "initialize reduced audit config: error getting user role memberships: %v", err)
return
}
// Get matching audit setting.
p.reducedAuditConfig.AuditSetting = p.AuditConfig().GetMatchingAuditSetting(userRoles, user)
}

// shouldNotRoleBasedAudit checks if we should do any auditing work for RoleBasedAuditEvents.
func (p *planner) shouldNotRoleBasedAudit() bool {
// Do not do audit work if the cluster setting is empty.
Expand Down
120 changes: 120 additions & 0 deletions pkg/sql/audit_logging_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,126 @@ func TestMultiRoleAuditLogging(t *testing.T) {
}
}

func TestReducedAuditConfig(t *testing.T) {
defer leaktest.AfterTest(t)()
sc := log.ScopeWithoutShowLogs(t)
defer sc.Close(t)

cleanup := logtestutils.InstallLogFileSink(sc, t, logpb.Channel_SENSITIVE_ACCESS)
defer cleanup()

s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{})
rootRunner := sqlutils.MakeSQLRunner(sqlDB)
defer s.Stopper().Stop(context.Background())

// Dummy table/user used by tests.
setupQueries(t, rootRunner)

// Enable reduced config.
rootRunner.Exec(t, `SET CLUSTER SETTING sql.log.user_audit.reduced_config.enabled = true`)
testutils.SucceedsSoon(t, func() error {
var currentVal string
rootRunner.QueryRow(t,
"SHOW CLUSTER SETTING sql.log.user_audit.reduced_config.enabled",
).Scan(&currentVal)

if currentVal == "false" {
return errors.Newf("waiting for reduced config cluster setting to be true")
}
return nil
})

testUserURL, cleanupFn := sqlutils.PGUrl(t,
s.ServingSQLAddr(), t.Name(), url.User(username.TestUser))
defer cleanupFn()

testUserDb, err := gosql.Open("postgres", testUserURL.String())
require.NoError(t, err)
defer testUserDb.Close()
testRunner := sqlutils.MakeSQLRunner(testUserDb)

// Set a cluster configuration.
roleA := "roleA"
rootRunner.Exec(t, `SET CLUSTER SETTING sql.log.user_audit = '
roleA ALL
'`)

testutils.SucceedsSoon(t, func() error {
var currentVal string
rootRunner.QueryRow(t,
"SHOW CLUSTER SETTING sql.log.user_audit",
).Scan(&currentVal)

if currentVal == "" {
return errors.Newf("waiting for cluster setting to be set")
}
return nil
})

// Run a query. This initializes the reduced audit configuration for the user.
// Currently, there are no corresponding roles for the user in the audit configuration.
// Consequently, the user's reduced audit config will be nil.
testQuery := `SELECT * FROM u`
testRunner.Exec(t, testQuery)

// Grant a role the user that corresponds to an audit setting.
rootRunner.Exec(t, fmt.Sprintf("CREATE ROLE IF NOT EXISTS %s", roleA))
rootRunner.Exec(t, fmt.Sprintf("GRANT %s to testuser", roleA))

// Run the query again. We expect no log from this query even though the user now has a corresponding role
// as the reduced audit configuration has already been computed, and there were no corresponding audit settings
// for the user at that time.
testRunner.Exec(t, testQuery)

log.Flush()

entries, err := log.FetchEntriesFromFiles(
0,
math.MaxInt64,
10000,
regexp.MustCompile(`"EventType":"role_based_audit_event"`),
log.WithMarkedSensitiveData,
)

if err != nil {
t.Fatal(err)
}

// Expect the number of entries to be 0.
if len(entries) != 0 {
t.Fatal(errors.Newf("unexpected entries found"))
}

// Open 2nd connection for the test user.
testUserDb2, err := gosql.Open("postgres", testUserURL.String())
require.NoError(t, err)
defer testUserDb2.Close()
testRunner2 := sqlutils.MakeSQLRunner(testUserDb2)

// Run a query on the new connection. The new connection will cause the reduced audit config to be re-computed.
// The user now has a corresponding audit setting. We use a new query here to differentiate.
testRunner2.Exec(t, `GRANT SELECT ON TABLE u TO root`)

log.Flush()

entries, err = log.FetchEntriesFromFiles(
0,
math.MaxInt64,
10000,
regexp.MustCompile(`GRANT SELECT ON TABLE ‹u› TO root`),
log.WithMarkedSensitiveData,
)

if err != nil {
t.Fatal(err)
}

// Expect the number of entries to be 1.
if len(entries) != 1 {
t.Fatal(errors.Newf("unexpected number of entries found (not 1)"))
}
}

func setupQueries(t *testing.T, rootRunner *sqlutils.SQLRunner) {
// Dummy table/user used by tests.
rootRunner.Exec(t, `CREATE TABLE u(x int)`)
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/auditlogging/audit_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,13 @@ type AuditConfigLock struct {
Config *AuditConfig
}

// ReducedAuditConfig is a computed audit configuration initialized at the first SQL event emit by the user.
type ReducedAuditConfig struct {
syncutil.RWMutex
Initialized bool
AuditSetting *AuditSetting
}

// GetMatchingAuditSetting returns the first audit setting in the cluster setting
// configuration that matches the user's username/roles. If no such audit setting
// exists, returns nil.
Expand Down
19 changes: 18 additions & 1 deletion pkg/sql/auditlogging/audit_log_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,24 @@ var UserAuditLogConfig = settings.RegisterValidatedStringSetting(
"user/role-based audit logging configuration",
"",
validateAuditLogConfig,
)
).WithPublic()

// UserAuditEnableReducedConfig is a cluster setting that enables/disables a computed
// reduced configuration. This allows us to compute the audit configuration once at
// session start, instead of computing at each SQL event. The tradeoff is that changes to
// the audit configuration (user role memberships or cluster setting configuration) are not
// reflected within session. Users will need to start a new session to see these changes in their
// auditing behaviour.
var UserAuditEnableReducedConfig = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.log.user_audit.reduced_config.enabled",
"enables logic to compute a reduced audit configuration, computing the audit "+
"configuration only once at session start instead of at each SQL event. The tradeoff "+
"with the increase in performance (~5%), is that changes to the audit configuration "+
"(user role memberships/cluster setting) are not reflected within session. "+
"Users will need to start a new session to see these changes in their auditing behaviour.",
false,
).WithPublic()

func validateAuditLogConfig(_ *settings.Values, input string) error {
if input == "" {
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/appstatspb"
"github.com/cockroachdb/cockroach/pkg/sql/auditlogging"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catsessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descidgen"
Expand Down Expand Up @@ -3575,6 +3576,7 @@ func (ex *connExecutor) initPlanner(ctx context.Context, p *planner) {
p.schemaResolver.sessionDataStack = p.EvalContext().SessionDataStack
p.schemaResolver.descCollection = p.Descriptors()
p.schemaResolver.authAccessor = p
p.reducedAuditConfig = &auditlogging.ReducedAuditConfig{}
}

func (ex *connExecutor) resetPlanner(
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -271,6 +271,8 @@ type planner struct {

// trackDependency is used to track circular dependencies when dropping views.
trackDependency map[catid.DescID]bool

reducedAuditConfig *auditlogging.ReducedAuditConfig
}

// hasFlowForPausablePortal returns true if the planner is for re-executing a
Expand Down

0 comments on commit 530e2ef

Please sign in to comment.