diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index b1bd30c09011..e638c178b231 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -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 diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index a7524e73c0ba..b1c96905db16 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", @@ -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", @@ -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", 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"},