From f07d33879616c669a929f1802aebcef8dc47a00a Mon Sep 17 00:00:00 2001 From: Thomas Hardy Date: Mon, 5 Jun 2023 23:02:04 -0400 Subject: [PATCH] auditloggingccl: migrate role-based audit logging as a CCL feature 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 - 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 cluster setting's validation function. Users will be unable to change this cluster setting (and thereby will not be able to enable role-based audit logging) unless they have an enterprise CCL license - the addition of function hooks to be used on CCL initialization, namely: - `ConfigureRoleBasedAuditClusterSettings`: used to add the `SetOnChange` cluster setting hook to the audit logging cluster setting - `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. --- .github/CODEOWNERS | 1 + pkg/BUILD.bazel | 4 ++ pkg/ccl/BUILD.bazel | 1 + pkg/ccl/auditloggingccl/BUILD.bazel | 52 +++++++++++++++ .../auditloggingccl}/audit_log_config.go | 65 +++++++++++++++---- .../auditloggingccl}/audit_logging_test.go | 31 +++++++-- pkg/ccl/auditloggingccl/main_test.go | 31 +++++++++ pkg/ccl/ccl_init.go | 1 + pkg/server/server_sql.go | 12 ++-- pkg/sql/BUILD.bazel | 1 - pkg/sql/audit_logging.go | 4 +- pkg/sql/auditlogging/BUILD.bazel | 3 +- pkg/sql/auditlogging/audit_log.go | 41 +++++++++--- pkg/sql/auditlogging/audit_log_test.go | 2 +- pkg/sql/auditlogging/parser.go | 8 +-- pkg/sql/auditlogging/testdata/parse | 4 +- 16 files changed, 214 insertions(+), 47 deletions(-) create mode 100644 pkg/ccl/auditloggingccl/BUILD.bazel rename pkg/{sql/auditlogging => ccl/auditloggingccl}/audit_log_config.go (54%) rename pkg/{sql => ccl/auditloggingccl}/audit_logging_test.go (90%) create mode 100644 pkg/ccl/auditloggingccl/main_test.go diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index fd60c4d7d770..2c3c3f807fde 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -523,6 +523,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 diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 6650b21bbcf9..12d5ccb85be7 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -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", @@ -742,6 +743,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", @@ -2447,6 +2450,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", diff --git a/pkg/ccl/BUILD.bazel b/pkg/ccl/BUILD.bazel index 26204b3514da..e02d069e64f3 100644 --- a/pkg/ccl/BUILD.bazel +++ b/pkg/ccl/BUILD.bazel @@ -8,6 +8,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/ccl/auditloggingccl", "//pkg/ccl/backupccl", "//pkg/ccl/buildccl", "//pkg/ccl/changefeedccl", diff --git a/pkg/ccl/auditloggingccl/BUILD.bazel b/pkg/ccl/auditloggingccl/BUILD.bazel new file mode 100644 index 000000000000..24e8eb70e992 --- /dev/null +++ b/pkg/ccl/auditloggingccl/BUILD.bazel @@ -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") diff --git a/pkg/sql/auditlogging/audit_log_config.go b/pkg/ccl/auditloggingccl/audit_log_config.go similarity index 54% rename from pkg/sql/auditlogging/audit_log_config.go rename to pkg/ccl/auditloggingccl/audit_log_config.go index d70539bfe474..f6f28a3b296f 100644 --- a/pkg/sql/auditlogging/audit_log_config.go +++ b/pkg/ccl/auditloggingccl/audit_log_config.go @@ -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() @@ -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 ('').") @@ -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 +} diff --git a/pkg/sql/audit_logging_test.go b/pkg/ccl/auditloggingccl/audit_logging_test.go similarity index 90% rename from pkg/sql/audit_logging_test.go rename to pkg/ccl/auditloggingccl/audit_logging_test.go index 20f0d098fdf5..1c34fbca642d 100644 --- a/pkg/sql/audit_logging_test.go +++ b/pkg/ccl/auditloggingccl/audit_logging_test.go @@ -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" @@ -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" @@ -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) diff --git a/pkg/ccl/auditloggingccl/main_test.go b/pkg/ccl/auditloggingccl/main_test.go new file mode 100644 index 000000000000..cb440e0111b4 --- /dev/null +++ b/pkg/ccl/auditloggingccl/main_test.go @@ -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()) +} diff --git a/pkg/ccl/ccl_init.go b/pkg/ccl/ccl_init.go index a56214fd4a70..f266b2ec31d3 100644 --- a/pkg/ccl/ccl_init.go +++ b/pkg/ccl/ccl_init.go @@ -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" diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 279858e52ac0..b127ea0111be 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -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, diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 39bf547bf67c..00f2ed584618 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -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", diff --git a/pkg/sql/audit_logging.go b/pkg/sql/audit_logging.go index 951bbb87edf4..2452093b150b 100644 --- a/pkg/sql/audit_logging.go +++ b/pkg/sql/audit_logging.go @@ -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 } @@ -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 } diff --git a/pkg/sql/auditlogging/BUILD.bazel b/pkg/sql/auditlogging/BUILD.bazel index fe5c22e0a7c1..f51e7213fb10 100644 --- a/pkg/sql/auditlogging/BUILD.bazel +++ b/pkg/sql/auditlogging/BUILD.bazel @@ -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", @@ -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", ], diff --git a/pkg/sql/auditlogging/audit_log.go b/pkg/sql/auditlogging/audit_log.go index 47eb1b380678..74f6352d64fd 100644 --- a/pkg/sql/auditlogging/audit_log.go +++ b/pkg/sql/auditlogging/audit_log.go @@ -17,13 +17,36 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" "github.com/olekukonko/tablewriter" ) +// ConfigureRoleBasedAuditClusterSettings is a noop global var injected by CCL hook. +// See corresponding ConfigureRoleBasedAuditClusterSettings in auditloggingccl. +var ConfigureRoleBasedAuditClusterSettings = func(ctx context.Context, acl *AuditConfigLock, st *cluster.Settings, sv *settings.Values) { +} + +// UserAuditLogConfigEmpty is a noop global var injected by CCL hook. +var UserAuditLogConfigEmpty = func(sv *settings.Values) bool { + return true +} + +// UserAuditReducedConfigEnabled is a noop global var injected by CCL hook. +var UserAuditReducedConfigEnabled = func(sv *settings.Values) bool { + return false +} + +var UserAuditEnterpriseParamsHook = func() (*cluster.Settings, uuid.UUID, error) { + return nil, uuid.Nil, errors.New("Cannot validate log config, enterprise parameters not initialized yet") +} + // Auditor is an interface used to check and build different audit events. type Auditor interface { GetQualifiedTableNameByID(ctx context.Context, id int64, requiredType tree.RequiredTableKind) (*tree.TableName, error) @@ -74,16 +97,16 @@ func (cl *AuditConfigLock) GetMatchingAuditSetting( // AuditConfig is a parsed configuration. type AuditConfig struct { - // settings are the collection of AuditSettings that make up the AuditConfig. - settings []AuditSetting - // allRoleAuditSettingIdx is an index corresponding to an AuditSetting in settings that applies to all + // Settings are the collection of AuditSettings that make up the AuditConfig. + Settings []AuditSetting + // allRoleAuditSettingIdx is an index corresponding to an AuditSetting in Settings that applies to all // users, if it exists. Default value -1 (defaultAllAuditSettingIdx). allRoleAuditSettingIdx int } const defaultAllAuditSettingIdx = -1 -// EmptyAuditConfig returns an audit configuration with no audit settings. +// EmptyAuditConfig returns an audit configuration with no audit Settings. func EmptyAuditConfig() *AuditConfig { return &AuditConfig{ allRoleAuditSettingIdx: defaultAllAuditSettingIdx, @@ -96,7 +119,7 @@ func (c AuditConfig) getMatchingAuditSetting( userRoles map[username.SQLUsername]bool, name username.SQLUsername, ) *AuditSetting { // If the user matches any Setting, return the corresponding filter. - for idx, filter := range c.settings { + for idx, filter := range c.Settings { // If we have matched an audit setting by role, return the audit setting. _, exists := userRoles[filter.Role] if exists { @@ -116,13 +139,13 @@ func (c AuditConfig) getMatchingAuditSetting( } func (c AuditConfig) String() string { - if len(c.settings) == 0 { + if len(c.Settings) == 0 { return "# (empty configuration)\n" } var sb strings.Builder sb.WriteString("# Original configuration:\n") - for _, setting := range c.settings { + for _, setting := range c.Settings { fmt.Fprintf(&sb, "# %s\n", setting.input) } sb.WriteString("#\n# Interpreted configuration:\n") @@ -138,7 +161,7 @@ func (c AuditConfig) String() string { row := []string{"# ROLE", "STATEMENT_FILTER"} table.Append(row) - for _, setting := range c.settings { + for _, setting := range c.Settings { row[0] = setting.Role.Normalized() row[1] = writeStatementFilter(setting.IncludeStatements) table.Append(row) @@ -167,5 +190,5 @@ type AuditSetting struct { } func (s AuditSetting) String() string { - return AuditConfig{settings: []AuditSetting{s}}.String() + return AuditConfig{Settings: []AuditSetting{s}}.String() } diff --git a/pkg/sql/auditlogging/audit_log_test.go b/pkg/sql/auditlogging/audit_log_test.go index ffc72fe4343f..8044d1e0d546 100644 --- a/pkg/sql/auditlogging/audit_log_test.go +++ b/pkg/sql/auditlogging/audit_log_test.go @@ -26,7 +26,7 @@ func TestParse(t *testing.T) { func(t *testing.T, td *datadriven.TestData) string { switch td.Cmd { case "multiline": - config, err := parse(td.Input) + config, err := Parse(td.Input) if err != nil { return fmt.Sprintf("error: %v\n", err) } diff --git a/pkg/sql/auditlogging/parser.go b/pkg/sql/auditlogging/parser.go index 9a1212470a97..bf84a50eb81f 100644 --- a/pkg/sql/auditlogging/parser.go +++ b/pkg/sql/auditlogging/parser.go @@ -20,15 +20,15 @@ import ( "github.com/cockroachdb/errors" ) -// parse parses the provided audit logging configuration. -func parse(input string) (*AuditConfig, error) { +// Parse parses the provided audit logging configuration. +func Parse(input string) (*AuditConfig, error) { tokens, err := rulebasedscanner.Tokenize(input) if err != nil { return nil, err } config := EmptyAuditConfig() - config.settings = make([]AuditSetting, len(tokens.Lines)) + config.Settings = make([]AuditSetting, len(tokens.Lines)) // settingsRoleMap keeps track of the roles we've already written in the config settingsRoleMap := make(map[username.SQLUsername]interface{}, len(tokens.Lines)) for i, line := range tokens.Lines { @@ -42,7 +42,7 @@ func parse(input string) (*AuditConfig, error) { return nil, errors.Newf("duplicate role listed: %v", setting.Role) } settingsRoleMap[setting.Role] = i - config.settings[i] = setting + config.Settings[i] = setting if setting.Role.Normalized() == allUserRole { config.allRoleAuditSettingIdx = i } diff --git a/pkg/sql/auditlogging/testdata/parse b/pkg/sql/auditlogging/testdata/parse index 2d9cb5f3ba57..6ecc9c4a0226 100644 --- a/pkg/sql/auditlogging/testdata/parse +++ b/pkg/sql/auditlogging/testdata/parse @@ -189,7 +189,7 @@ anotherrole ALL thirddrole NONE # Detail: &auditlogging.AuditConfig{ - settings: { + Settings: { { input: "test_role ALL", Role: username.SQLUsername{u:"test_role"}, @@ -231,7 +231,7 @@ all ALL thirddrole ALL # Detail: &auditlogging.AuditConfig{ - settings: { + Settings: { { input: "test_role NONE", Role: username.SQLUsername{u:"test_role"},