Skip to content

Commit

Permalink
Merge #104383
Browse files Browse the repository at this point in the history
104383: auditloggingccl: migrate role-based audit logging as a CCL feature r=THardy98 a=THardy98

Part of: #100467

This change moves the existing role-based audit logging logic to be
consumed as a CCL (enterprise) feature.

The notable changes here are:
- moving the audit logging cluster settings (i.e. `sql.log.user_audit`
  and `sql.log.user_audit.reduced_config.enabled`) to the `ccl` package.
Consequently, these cluster settings will only exist when the use has a
CCL license (free or paid)
- gating the `sql.log.user_audit` cluster setting behind the enterprise
  CCL license. This was done by adding an enterprise CCL license check
in the `sql.log.user_audit` validation function. Users will be unable to
change this cluster setting (and thereby will not be able to enable/configure
role-based audit logging) unless they have an enterprise CCL license
-  the addition of function hooks to be used at CCL initialization,
   namely:
   - `ConfigureRoleBasedAuditClusterSettings`: used to add a
     `SetOnChange` hook to `sql.log.user_audit`
   - `UserAuditLogConfigEmpty`: used to check whether the audit logging
     cluster setting is empty
   - `UserAuditReducedConfigEnabled`: used to check whether the reduced
     audit configuration is enabled (note: regular users are still able
to enable/disable this cluster setting, but it will take no effect as
they will have no way to enable role-based audit logging)
   - `UserAuditEnterpriseParamsHook`: used to retrieve parameters
     necessary for enterprise license checks within the the
`sql.log.user_audit` cluster setting validation function

Release note (sql change): Role-based audit logging is now a CCL
(enterprise) feature. Only enterprise CCL users will be able to
configure role-based audit logging using the `sql.log.user_audit`
cluster setting.

Co-authored-by: Thomas Hardy <[email protected]>
  • Loading branch information
craig[bot] and Thomas Hardy committed Jun 8, 2023
2 parents 46cc1a6 + f07d338 commit 1824811
Show file tree
Hide file tree
Showing 16 changed files with 214 additions and 47 deletions.
1 change: 1 addition & 0 deletions .github/CODEOWNERS
Original file line number Diff line number Diff line change
Expand Up @@ -524,6 +524,7 @@
/pkg/workload/ @cockroachdb/test-eng #! @cockroachdb/sql-foundations-noreview
/pkg/obs/ @cockroachdb/obs-inf-prs
/pkg/obsservice/ @cockroachdb/obs-inf-prs
/pkg/ccl/auditloggingccl @cockroachdb/cluster-observability

# Own all bazel files to dev-inf, but don't request reviews for them
# as they are mostly - but not only - generated code that changes with
Expand Down
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ ALL_TESTS = [
"//pkg/build/starlarkutil:starlarkutil_test",
"//pkg/build/util:util_test",
"//pkg/build:build_test",
"//pkg/ccl/auditloggingccl:auditloggingccl_test",
"//pkg/ccl/backupccl/backupdest:backupdest_test",
"//pkg/ccl/backupccl/backupinfo:backupinfo_disallowed_imports_test",
"//pkg/ccl/backupccl/backupinfo:backupinfo_test",
Expand Down Expand Up @@ -741,6 +742,8 @@ GO_TARGETS = [
"//pkg/build/util:util_test",
"//pkg/build:build",
"//pkg/build:build_test",
"//pkg/ccl/auditloggingccl:auditloggingccl",
"//pkg/ccl/auditloggingccl:auditloggingccl_test",
"//pkg/ccl/backupccl/backupbase:backupbase",
"//pkg/ccl/backupccl/backupdest:backupdest",
"//pkg/ccl/backupccl/backupdest:backupdest_test",
Expand Down Expand Up @@ -2449,6 +2452,7 @@ GET_X_DATA_TARGETS = [
"//pkg/build/starlarkutil:get_x_data",
"//pkg/build/util:get_x_data",
"//pkg/ccl:get_x_data",
"//pkg/ccl/auditloggingccl:get_x_data",
"//pkg/ccl/backupccl:get_x_data",
"//pkg/ccl/backupccl/backupbase:get_x_data",
"//pkg/ccl/backupccl/backupdest:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/ccl/auditloggingccl",
"//pkg/ccl/backupccl",
"//pkg/ccl/buildccl",
"//pkg/ccl/changefeedccl",
Expand Down
52 changes: 52 additions & 0 deletions pkg/ccl/auditloggingccl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "auditloggingccl",
srcs = ["audit_log_config.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/auditloggingccl",
visibility = ["//visibility:public"],
deps = [
"//pkg/ccl/utilccl",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/auditlogging",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/util/log",
"@com_github_cockroachdb_errors//:errors",
],
)

go_test(
name = "auditloggingccl_test",
srcs = [
"audit_logging_test.go",
"main_test.go",
],
args = ["-test.timeout=295s"],
embed = [":auditloggingccl"],
tags = ["ccl_test"],
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/ccl/utilccl",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/security/username",
"//pkg/server",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/log/logpb",
"//pkg/util/log/logtestutils",
"//pkg/util/randutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_stretchr_testify//require",
],
)

get_x_data(name = "get_x_data")
Original file line number Diff line number Diff line change
@@ -1,30 +1,34 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// 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.
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package auditlogging
package auditloggingccl

import (
"context"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/auditlogging"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

const auditConfigDefaultValue = ""

// UserAuditLogConfig is a cluster setting that takes a user/role-based audit configuration.
var UserAuditLogConfig = settings.RegisterValidatedStringSetting(
settings.TenantWritable,
"sql.log.user_audit",
"user/role-based audit logging configuration",
"",
auditConfigDefaultValue,
validateAuditLogConfig,
).WithPublic()

Expand All @@ -46,16 +50,25 @@ var UserAuditEnableReducedConfig = settings.RegisterBoolSetting(
).WithPublic()

func validateAuditLogConfig(_ *settings.Values, input string) error {
if input == "" {
if input == auditConfigDefaultValue {
// Empty config
return nil
}
st, clusterID, err := auditlogging.UserAuditEnterpriseParamsHook()
if err != nil {
return err
}
enterpriseCheckErr := utilccl.CheckEnterpriseEnabled(st, clusterID, "role-based audit logging")
if enterpriseCheckErr != nil {
return pgerror.Wrap(enterpriseCheckErr,
pgcode.InsufficientPrivilege, "role-based audit logging requires enterprise license")
}
// Ensure it can be parsed.
conf, err := parse(input)
conf, err := auditlogging.Parse(input)
if err != nil {
return err
}
if len(conf.settings) == 0 {
if len(conf.Settings) == 0 {
// The string was not empty, but we were unable to parse any settings.
return errors.WithHint(errors.New("no entries"),
"To use the default configuration, assign the empty string ('').")
Expand All @@ -66,17 +79,41 @@ func validateAuditLogConfig(_ *settings.Values, input string) error {
// UpdateAuditConfigOnChange initializes the local
// node's audit configuration each time the cluster setting
// is updated.
func UpdateAuditConfigOnChange(ctx context.Context, acl *AuditConfigLock, st *cluster.Settings) {
func UpdateAuditConfigOnChange(
ctx context.Context, acl *auditlogging.AuditConfigLock, st *cluster.Settings,
) {
val := UserAuditLogConfig.Get(&st.SV)
config, err := parse(val)
config, err := auditlogging.Parse(val)
if err != nil {
// We encounter an error parsing (i.e. invalid config), fallback
// to an empty config.
log.Ops.Warningf(ctx, "invalid audit log config (sql.log.user_audit): %v\n"+
"falling back to empty audit config", err)
config = EmptyAuditConfig()
config = auditlogging.EmptyAuditConfig()
}
acl.Lock()
acl.Config = config
acl.Unlock()
}

var ConfigureRoleBasedAuditClusterSettings = func(ctx context.Context, acl *auditlogging.AuditConfigLock, st *cluster.Settings, sv *settings.Values) {
UserAuditLogConfig.SetOnChange(
sv, func(ctx context.Context) {
UpdateAuditConfigOnChange(ctx, acl, st)
})
UpdateAuditConfigOnChange(ctx, acl, st)
}

var UserAuditLogConfigEmpty = func(sv *settings.Values) bool {
return UserAuditLogConfig.Get(sv) == ""
}

var UserAuditReducedConfigEnabled = func(sv *settings.Values) bool {
return UserAuditEnableReducedConfig.Get(sv)
}

func init() {
auditlogging.ConfigureRoleBasedAuditClusterSettings = ConfigureRoleBasedAuditClusterSettings
auditlogging.UserAuditLogConfigEmpty = UserAuditLogConfigEmpty
auditlogging.UserAuditReducedConfigEnabled = UserAuditReducedConfigEnabled
}
Original file line number Diff line number Diff line change
@@ -1,14 +1,12 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// 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.
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package sql
package auditloggingccl

import (
"context"
Expand All @@ -21,6 +19,7 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand All @@ -33,6 +32,24 @@ import (
"github.com/stretchr/testify/require"
)

func TestRoleBasedAuditEnterpriseGated(t *testing.T) {
defer leaktest.AfterTest(t)()

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

testQuery := `SET CLUSTER SETTING sql.log.user_audit = 'ALL ALL'`

// Test that we cannot change the cluster setting when enterprise is disabled.
enableEnterprise := utilccl.TestingDisableEnterprise()
rootRunner.ExpectErr(t, "role-based audit logging requires enterprise license", testQuery)
// Enable enterprise.
enableEnterprise()
// Test that we can change the cluster setting when enterprise is enabled.
rootRunner.Exec(t, testQuery)
}

func TestSingleRoleAuditLogging(t *testing.T) {
defer leaktest.AfterTest(t)()
sc := log.ScopeWithoutShowLogs(t)
Expand Down
31 changes: 31 additions & 0 deletions pkg/ccl/auditloggingccl/main_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package auditloggingccl_test

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

func TestMain(m *testing.M) {
defer ccl.TestingEnableEnterprise()()
securityassets.SetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)
os.Exit(m.Run())
}
1 change: 1 addition & 0 deletions pkg/ccl/ccl_init.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package ccl

import (
"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl/auditloggingccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/backupccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/buildccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl"
Expand Down
12 changes: 7 additions & 5 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1374,11 +1374,13 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
vmoduleSetting.SetOnChange(&cfg.Settings.SV, fn)
fn(ctx)

auditlogging.UserAuditLogConfig.SetOnChange(
&execCfg.Settings.SV, func(ctx context.Context) {
auditlogging.UpdateAuditConfigOnChange(ctx, execCfg.SessionInitCache.AuditConfig, execCfg.Settings)
})
auditlogging.UpdateAuditConfigOnChange(ctx, execCfg.SessionInitCache.AuditConfig, execCfg.Settings)
auditlogging.UserAuditEnterpriseParamsHook = func(st *cluster.Settings, clusterID uuid.UUID) func() (*cluster.Settings, uuid.UUID, error) {
return func() (*cluster.Settings, uuid.UUID, error) {
return st, clusterID, nil
}
}(execCfg.Settings, cfg.ClusterIDContainer.Get())

auditlogging.ConfigureRoleBasedAuditClusterSettings(ctx, execCfg.SessionInitCache.AuditConfig, execCfg.Settings, &execCfg.Settings.SV)

return &SQLServer{
ambientCtx: cfg.BaseConfig.AmbientCtx,
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -593,7 +593,6 @@ go_test(
"alter_column_type_test.go",
"ambiguous_commit_test.go",
"as_of_test.go",
"audit_logging_test.go",
"authorization_test.go",
"backfill_num_ranges_in_span_test.go",
"backfill_protected_timestamp_test.go",
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/audit_logging.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ func (p *planner) maybeAuditRoleBasedAuditEvent(ctx context.Context) {
}

// Use reduced audit config is enabled.
if auditlogging.UserAuditEnableReducedConfig.Get(&p.execCfg.Settings.SV) {
if auditlogging.UserAuditReducedConfigEnabled(&p.execCfg.Settings.SV) {
p.logReducedAuditConfig(ctx)
return
}
Expand Down Expand Up @@ -119,5 +119,5 @@ func (p *planner) shouldNotRoleBasedAudit() bool {
// Do not do audit work if the cluster setting is empty.
// Do not emit audit events for reserved users/roles. This does not omit the root user.
// Do not emit audit events for internal planners.
return auditlogging.UserAuditLogConfig.Get(&p.execCfg.Settings.SV) == "" || p.User().IsReserved() || p.isInternalPlanner
return auditlogging.UserAuditLogConfigEmpty(&p.execCfg.Settings.SV) || p.User().IsReserved() || p.isInternalPlanner
}
3 changes: 1 addition & 2 deletions pkg/sql/auditlogging/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@ go_library(
name = "auditlogging",
srcs = [
"audit_log.go",
"audit_log_config.go",
"parser.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/auditlogging",
Expand All @@ -19,10 +18,10 @@ go_library(
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/tree",
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/log/logpb",
"//pkg/util/syncutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_olekukonko_tablewriter//:tablewriter",
],
Expand Down
Loading

0 comments on commit 1824811

Please sign in to comment.