From ea755eb1e841904c4b92ef88ace9a51faef1d5d8 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Tue, 27 Jul 2021 10:43:13 -0400 Subject: [PATCH 1/8] sql: settings from ALTER ROLE ... SET apply on session init The default settings are applied when creating a new pgwire session. The information is cached alongside authentication info to reduce the latency of establishing a connection and minimize the chance of a login availability hit. Release note (sql change): Default session variable settings configured by `ALTER ROLE ... SET` are now supported. The following order of precedence is used for variable settings: 1. settings specified in the connection URL as a query parameter. 2. per-role and per-database setting configured by ALTER ROLE. 3. per-role and all-database setting configured by ALTER ROLE. 4. all-role and per-database setting configured by ALTER ROLE. 5. all-role and all-database setting configured by ALTER ROLE. RESET does not validate the setting name. SET validates both the name and the proposed default value. Note that the default settings for a role are not inherited if one role is a member of another role that has default settings. Also, the defaults _only_ apply during session initialization. Using `SET DATABASE` to change databases does not apply default settings for that database. The `public`, `admin`, and `root` roles cannot have default session variables configured. The `root` role also will never use the "all-role" default settings. This is so that `root` has fewer dependencies during session initialization and to make it less likely for `root` authentication to become unavailable during the loss of a node. Changing the default settings for a role requires the role running the ALTER command to either be an ADMIN or to have the CREATEROLE role option. Only ADMINs can edit the default settings for another admin. Futhermore, changing the default settings for `ALL` roles is _only_ allowed for ADMINs. Roles without ADMIN or CREATEROLE _cannot_ change the default settings for themselves. --- pkg/ccl/gssapiccl/BUILD.bazel | 1 + pkg/ccl/gssapiccl/gssapi.go | 3 +- pkg/ccl/serverccl/role_authentication_test.go | 14 +- pkg/server/authentication.go | 10 +- pkg/sql/alter_role.go | 45 ++- pkg/sql/authentication/BUILD.bazel | 1 + pkg/sql/authentication/cache.go | 322 ++++++++++++++---- pkg/sql/drop_database.go | 13 +- pkg/sql/drop_role.go | 16 +- pkg/sql/pgwire/auth.go | 30 +- pkg/sql/pgwire/auth_methods.go | 26 +- pkg/sql/pgwire/conn_test.go | 150 ++++++++ pkg/sql/user.go | 185 +++++++--- pkg/sql/vars.go | 25 ++ 14 files changed, 665 insertions(+), 176 deletions(-) diff --git a/pkg/ccl/gssapiccl/BUILD.bazel b/pkg/ccl/gssapiccl/BUILD.bazel index 8f10ef8a4c91..496841fb904c 100644 --- a/pkg/ccl/gssapiccl/BUILD.bazel +++ b/pkg/ccl/gssapiccl/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/ccl/utilccl", "//pkg/security", "//pkg/sql", + "//pkg/sql/sem/tree", "//pkg/sql/pgwire", "//pkg/sql/pgwire/hba", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/ccl/gssapiccl/gssapi.go b/pkg/ccl/gssapiccl/gssapi.go index 19ea7388b04d..b1a34c67c212 100644 --- a/pkg/ccl/gssapiccl/gssapi.go +++ b/pkg/ccl/gssapiccl/gssapi.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -46,7 +47,7 @@ func authGSS( c pgwire.AuthConn, tlsState tls.ConnectionState, _ pgwire.PasswordRetrievalFn, - _ pgwire.PasswordValidUntilFn, + _ *tree.DTimestamp, execCfg *sql.ExecutorConfig, entry *hba.Entry, ) (security.UserAuthHook, error) { diff --git a/pkg/ccl/serverccl/role_authentication_test.go b/pkg/ccl/serverccl/role_authentication_test.go index 14c98e3c1591..649a7b116b30 100644 --- a/pkg/ccl/serverccl/role_authentication_test.go +++ b/pkg/ccl/serverccl/role_authentication_test.go @@ -115,7 +115,9 @@ func TestVerifyPassword(t *testing.T) { t.Run("", func(t *testing.T) { execCfg := s.ExecutorConfig().(sql.ExecutorConfig) username := security.MakeSQLUsernameFromPreNormalizedString(tc.username) - exists, canLogin, pwRetrieveFn, validUntilFn, err := sql.GetUserHashedPassword(context.Background(), &execCfg, &ie, username) + exists, canLogin, validUntil, _, pwRetrieveFn, err := sql.GetUserAuthInfo( + context.Background(), &execCfg, &ie, username, "", /* databaseName */ + ) if err != nil { t.Errorf( @@ -148,16 +150,6 @@ func TestVerifyPassword(t *testing.T) { valid = false } - validUntil, err := validUntilFn(ctx) - if err != nil { - t.Errorf( - "credentials %s/%s failed with error %s, wanted no error", - tc.username, - tc.password, - err, - ) - } - if validUntil != nil { if validUntil.Time.Sub(timeutil.Now()) < 0 { expired = true diff --git a/pkg/server/authentication.go b/pkg/server/authentication.go index 385421a9ea40..b06ec9bf29a0 100644 --- a/pkg/server/authentication.go +++ b/pkg/server/authentication.go @@ -254,11 +254,12 @@ func (s *authenticationServer) UserLoginFromSSO( // without further normalization. username, _ := security.MakeSQLUsernameFromUserInput(reqUsername, security.UsernameValidation) - exists, canLogin, _, _, err := sql.GetUserHashedPassword( + exists, canLogin, _, _, _, err := sql.GetUserAuthInfo( ctx, s.server.sqlServer.execCfg, s.server.sqlServer.execCfg.InternalExecutor, username, + "", /* databaseName */ ) if err != nil { @@ -416,11 +417,12 @@ WHERE id = $1` func (s *authenticationServer) verifyPassword( ctx context.Context, username security.SQLUsername, password string, ) (valid bool, expired bool, err error) { - exists, canLogin, pwRetrieveFn, validUntilFn, err := sql.GetUserHashedPassword( + exists, canLogin, validUntil, _, pwRetrieveFn, err := sql.GetUserAuthInfo( ctx, s.server.sqlServer.execCfg, s.server.sqlServer.execCfg.InternalExecutor, username, + "", /* databaseName */ ) if err != nil { return false, false, err @@ -433,10 +435,6 @@ func (s *authenticationServer) verifyPassword( return false, false, err } - validUntil, err := validUntilFn(ctx) - if err != nil { - return false, false, err - } if validUntil != nil { if validUntil.Time.Sub(timeutil.Now()) < 0 { return false, true, nil diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go index 940112c82531..d657cdce51a9 100644 --- a/pkg/sql/alter_role.go +++ b/pkg/sql/alter_role.go @@ -466,8 +466,10 @@ func (n *alterRoleSetNode) startExec(params runParams) error { // Instead of inserting an empty settings array, this function will make // sure the row is deleted instead. upsertOrDeleteFunc := func(newSettings []string) error { + var rowsAffected int + var internalExecErr error if newSettings == nil { - _, err = params.extendedEvalCtx.ExecCfg.InternalExecutor.ExecEx( + rowsAffected, internalExecErr = params.extendedEvalCtx.ExecCfg.InternalExecutor.ExecEx( params.ctx, opName, params.p.txn, @@ -476,11 +478,8 @@ func (n *alterRoleSetNode) startExec(params runParams) error { n.dbDescID, roleName, ) - if err != nil { - return err - } } else { - _, err = params.extendedEvalCtx.ExecCfg.InternalExecutor.ExecEx( + rowsAffected, internalExecErr = params.extendedEvalCtx.ExecCfg.InternalExecutor.ExecEx( params.ctx, opName, params.p.txn, @@ -490,11 +489,23 @@ func (n *alterRoleSetNode) startExec(params runParams) error { roleName, newSettings, ) - if err != nil { + } + if internalExecErr != nil { + return internalExecErr + } + + if rowsAffected > 0 && authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { + // Bump database_role_settings table versions to force a refresh of AuthInfo cache. + if err := params.p.bumpDatabaseRoleSettingsTableVersion(params.ctx); err != nil { return err } } - return nil + return params.p.logEvent(params.ctx, + 0, /* no target */ + &eventpb.AlterRole{ + RoleName: roleName.Normalized(), + Options: []string{roleoption.DEFAULTSETTINGS.String()}, + }) } if n.setVarKind == resetAllVars { @@ -521,16 +532,7 @@ func (n *alterRoleSetNode) startExec(params runParams) error { newSetting := fmt.Sprintf("%s=%s", n.varName, strVal) newSettings = append(newSettings, newSetting) - if err := upsertOrDeleteFunc(newSettings); err != nil { - return err - } - - return params.p.logEvent(params.ctx, - 0, /* no target */ - &eventpb.AlterRole{ - RoleName: roleName.Normalized(), - Options: []string{roleoption.DEFAULTSETTINGS.String()}, - }) + return upsertOrDeleteFunc(newSettings) } // getRoleName resolves the roleName and performs additional validation @@ -655,14 +657,7 @@ func (n *alterRoleSetNode) getSessionVarVal(params runParams) (string, error) { // Validate the new string value, but don't actually apply it to any real // session. - fakeSessionMutator := &sessionDataMutator{ - data: &sessiondata.SessionData{}, - defaults: SessionDefaults(map[string]string{}), - settings: params.ExecCfg().Settings, - paramStatusUpdater: &noopParamStatusUpdater{}, - setCurTxnReadOnly: func(bool) {}, - } - if err := n.sVar.Set(params.ctx, fakeSessionMutator, strVal); err != nil { + if err := CheckSessionVariableValueValid(params.ctx, params.ExecCfg().Settings, n.varName, strVal); err != nil { return "", err } return strVal, nil diff --git a/pkg/sql/authentication/BUILD.bazel b/pkg/sql/authentication/BUILD.bazel index 8268c2b883b1..d686684115f8 100644 --- a/pkg/sql/authentication/BUILD.bazel +++ b/pkg/sql/authentication/BUILD.bazel @@ -9,6 +9,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/authentication", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/kv", "//pkg/security", "//pkg/settings", diff --git a/pkg/sql/authentication/cache.go b/pkg/sql/authentication/cache.go index 339ef53d34d7..aba406393bf7 100644 --- a/pkg/sql/authentication/cache.go +++ b/pkg/sql/authentication/cache.go @@ -14,6 +14,7 @@ import ( "context" "unsafe" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings" @@ -41,14 +42,17 @@ var CacheEnabled = settings.RegisterBoolSetting( ).WithPublic() // AuthInfoCache is a shared cache for hashed passwords and other -// information used during user authentication. +// information used during user authentication and session initialization. type AuthInfoCache struct { syncutil.Mutex - usersTableVersion descpb.DescriptorVersion - roleOptionsTableVersion descpb.DescriptorVersion - boundAccount mon.BoundAccount + usersTableVersion descpb.DescriptorVersion + roleOptionsTableVersion descpb.DescriptorVersion + dbRoleSettingsTableVersion descpb.DescriptorVersion + boundAccount mon.BoundAccount // cache is a mapping from username to AuthInfo. cache map[security.SQLUsername]AuthInfo + // settingsCache is a mapping from (dbID, username) to default settings. + settingsCache map[SettingsCacheKey][]string } // AuthInfo contains data that is used to perform an authentication attempt. @@ -63,6 +67,19 @@ type AuthInfo struct { ValidUntil *tree.DTimestamp } +// SettingsCacheKey is the key used for the settingsCache. +type SettingsCacheKey struct { + DatabaseID descpb.ID + Username security.SQLUsername +} + +// SettingsCacheEntry represents an entry in the settingsCache. It is +// used so that the entries can be returned in a stable order. +type SettingsCacheEntry struct { + SettingsCacheKey + Settings []string +} + // NewCache initializes a new AuthInfoCache. func NewCache(account mon.BoundAccount) *AuthInfoCache { return &AuthInfoCache{ @@ -70,31 +87,35 @@ func NewCache(account mon.BoundAccount) *AuthInfoCache { } } -// Get consults the AuthInfoCache and returns the AuthInfo for the provided -// normalizedUsername. If the information is not in the cache, or if the -// underlying tables have changed since the cache was populated, then the -// readFromStore callback is used to load new data. +// Get consults the AuthInfoCache and returns the AuthInfo and list of +// SettingsCacheEntry for the provided username and databaseName. If the +// information is not in the cache, or if the underlying tables have changed +// since the cache was populated, then the readFromStore callback is used to +// load new data. func (a *AuthInfoCache) Get( ctx context.Context, settings *cluster.Settings, ie sqlutil.InternalExecutor, db *kv.DB, f *descs.CollectionFactory, - normalizedUsername security.SQLUsername, - readFromStore func( + username security.SQLUsername, + databaseName string, + readFromSystemTables func( ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, - normalizedUsername security.SQLUsername, - ) (AuthInfo, error), -) (aInfo AuthInfo, err error) { - if !CacheEnabled.Get(&settings.SV) { - return readFromStore(ctx, nil /* txn */, ie, normalizedUsername) - } + username security.SQLUsername, + databaseID descpb.ID, + fetchDefaultSettings bool, + ) (AuthInfo, []SettingsCacheEntry, error), +) (aInfo AuthInfo, settingsEntries []SettingsCacheEntry, err error) { + // TODO(rafi): remove this flag in v21.2. + fetchDefaultSettings := settings.Version.IsActive(ctx, clusterversion.DatabaseRoleSettings) + err = f.Txn(ctx, ie, db, func( ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ) (err error) { - var usersTableDesc, roleOptionsTableDesc catalog.TableDescriptor + var usersTableDesc, roleOptionsTableDesc, dbRoleSettingsTableDesc catalog.TableDescriptor _, usersTableDesc, err = descriptors.GetImmutableTableByName( ctx, txn, @@ -113,81 +134,246 @@ func (a *AuthInfoCache) Get( if err != nil { return err } - if usersTableDesc.IsUncommittedVersion() || roleOptionsTableDesc.IsUncommittedVersion() { - aInfo, err = readFromStore(ctx, txn, ie, normalizedUsername) + if fetchDefaultSettings { + _, dbRoleSettingsTableDesc, err = descriptors.GetImmutableTableByName( + ctx, + txn, + DatabaseRoleSettingsTableName, + tree.ObjectLookupFlagsWithRequired(), + ) + } + if err != nil { + return err + } + databaseID := descpb.ID(0) + if databaseName != "" { + dbDesc, err := descriptors.GetImmutableDatabaseByName(ctx, txn, databaseName, tree.DatabaseLookupFlags{}) + if err != nil { + return err + } + // If dbDesc is nil, the database name was not valid, but that should + // not cause a login-preventing error. + if dbDesc != nil { + databaseID = dbDesc.GetID() + } + } + + if usersTableDesc.IsUncommittedVersion() || + roleOptionsTableDesc.IsUncommittedVersion() || + (fetchDefaultSettings && dbRoleSettingsTableDesc.IsUncommittedVersion()) || + !CacheEnabled.Get(&settings.SV) { + aInfo, settingsEntries, err = readFromSystemTables( + ctx, + txn, + ie, + username, + databaseID, + fetchDefaultSettings, + ) if err != nil { return err } } usersTableVersion := usersTableDesc.GetVersion() roleOptionsTableVersion := roleOptionsTableDesc.GetVersion() + dbRoleSettingsTableVersion := descpb.DescriptorVersion(0) + if fetchDefaultSettings { + dbRoleSettingsTableVersion = dbRoleSettingsTableDesc.GetVersion() + } // We loop in case the table version changes while looking up // password or role options. for { // Check version and maybe clear cache while holding the mutex. var found bool - aInfo, found = func() (AuthInfo, bool) { - a.Lock() - defer a.Unlock() - if a.usersTableVersion != usersTableVersion { - // Update users table version and drop the map. - a.usersTableVersion = usersTableVersion - a.cache = make(map[security.SQLUsername]AuthInfo) - a.boundAccount.Empty(ctx) - } - if a.roleOptionsTableVersion != roleOptionsTableVersion { - // Update role_optiosn table version and drop the map. - a.roleOptionsTableVersion = roleOptionsTableVersion - a.cache = make(map[security.SQLUsername]AuthInfo) - a.boundAccount.Empty(ctx) - } - aInfo, found = a.cache[normalizedUsername] - return aInfo, found - }() + aInfo, settingsEntries, found = a.readFromCache(ctx, usersTableVersion, roleOptionsTableVersion, dbRoleSettingsTableVersion, username, databaseID) if found { return nil } - // Lookup memberships outside the lock. - aInfo, err = readFromStore(ctx, txn, ie, normalizedUsername) + // Lookup the data outside the lock. + aInfo, settingsEntries, err = readFromSystemTables( + ctx, + txn, + ie, + username, + databaseID, + fetchDefaultSettings, + ) if err != nil { return err } - finishedLoop := func() bool { - // Update membership. - a.Lock() - defer a.Unlock() - // Table version has changed while we were looking: unlock and start over. - if a.usersTableVersion != usersTableVersion { - usersTableVersion = a.usersTableVersion - return false - } - if a.roleOptionsTableVersion != roleOptionsTableVersion { - roleOptionsTableVersion = a.roleOptionsTableVersion - return false - } - // Table version remains the same: update map, unlock, return. - const sizeOfAuthInfo = int(unsafe.Sizeof(AuthInfo{})) - const sizeOfTimestamp = int(unsafe.Sizeof(tree.DTimestamp{})) - sizeOfEntry := len(normalizedUsername.Normalized()) + - sizeOfAuthInfo + len(aInfo.HashedPassword) + sizeOfTimestamp - if err := a.boundAccount.Grow(ctx, int64(sizeOfEntry)); err != nil { - // If there is no memory available to cache the entry, we can still - // proceed with authentication so that users are not locked out of - // the database. - log.Ops.Warningf(ctx, "no memory available to cache authentication info: %v", err) - } else { - a.cache[normalizedUsername] = aInfo - } - return true - }() + finishedLoop := a.writeBackToCache( + ctx, + &usersTableVersion, + &roleOptionsTableVersion, + &dbRoleSettingsTableVersion, + aInfo, + settingsEntries, + username, + ) if finishedLoop { return nil } } }) - return aInfo, err + return aInfo, settingsEntries, err +} + +func (a *AuthInfoCache) readFromCache( + ctx context.Context, + usersTableVersion descpb.DescriptorVersion, + roleOptionsTableVersion descpb.DescriptorVersion, + dbRoleSettingsTableVersion descpb.DescriptorVersion, + username security.SQLUsername, + databaseID descpb.ID, +) (AuthInfo, []SettingsCacheEntry, bool) { + a.Lock() + defer a.Unlock() + a.checkStaleness(ctx, usersTableVersion, roleOptionsTableVersion, dbRoleSettingsTableVersion) + ai, foundAuthInfo := a.cache[username] + foundAllDefaultSettings := true + var sEntries []SettingsCacheEntry + // Search through the cache for the settings entries we need. Note + // that GenerateSettingsCacheKeys goes in order of precedence. + for _, k := range GenerateSettingsCacheKeys(databaseID, username) { + s, ok := a.settingsCache[k] + if !ok { + foundAllDefaultSettings = false + break + } + sEntries = append(sEntries, SettingsCacheEntry{k, s}) + } + return ai, sEntries, foundAuthInfo && foundAllDefaultSettings +} + +func (a *AuthInfoCache) checkStaleness( + ctx context.Context, + usersTableVersion descpb.DescriptorVersion, + roleOptionsTableVersion descpb.DescriptorVersion, + dbRoleSettingsTableVersion descpb.DescriptorVersion, +) { + if a.usersTableVersion != usersTableVersion { + // Update users table version and drop the map. + a.usersTableVersion = usersTableVersion + a.cache = make(map[security.SQLUsername]AuthInfo) + a.settingsCache = make(map[SettingsCacheKey][]string) + a.boundAccount.Empty(ctx) + } + if a.roleOptionsTableVersion != roleOptionsTableVersion { + // Update role_options table version and drop the map. + a.roleOptionsTableVersion = roleOptionsTableVersion + a.cache = make(map[security.SQLUsername]AuthInfo) + a.settingsCache = make(map[SettingsCacheKey][]string) + a.boundAccount.Empty(ctx) + } + if a.dbRoleSettingsTableVersion != dbRoleSettingsTableVersion { + // Update database_role_settings table version and drop the map. + a.dbRoleSettingsTableVersion = dbRoleSettingsTableVersion + a.cache = make(map[security.SQLUsername]AuthInfo) + a.settingsCache = make(map[SettingsCacheKey][]string) + a.boundAccount.Empty(ctx) + } +} + +// writeBackToCache tries to put the fetched data into the cache, and returns +// true if it succeeded. If the underlying system tables have been modified +// since they were read, the cache is not updated, and the passed in table +// descriptor versions are all updated. +func (a *AuthInfoCache) writeBackToCache( + ctx context.Context, + usersTableVersion *descpb.DescriptorVersion, + roleOptionsTableVersion *descpb.DescriptorVersion, + dbRoleSettingsTableVersion *descpb.DescriptorVersion, + aInfo AuthInfo, + settingsEntries []SettingsCacheEntry, + username security.SQLUsername, +) bool { + return func() bool { + a.Lock() + defer a.Unlock() + // Table version has changed while we were looking: unlock and start over. + if a.usersTableVersion != *usersTableVersion || + a.roleOptionsTableVersion != *roleOptionsTableVersion || + a.dbRoleSettingsTableVersion != *dbRoleSettingsTableVersion { + *usersTableVersion = a.usersTableVersion + *roleOptionsTableVersion = a.roleOptionsTableVersion + *dbRoleSettingsTableVersion = a.dbRoleSettingsTableVersion + return false + } + + // Table version remains the same: update map, unlock, return. + const sizeOfUsername = int(unsafe.Sizeof(security.SQLUsername{})) + const sizeOfAuthInfo = int(unsafe.Sizeof(AuthInfo{})) + const sizeOfTimestamp = int(unsafe.Sizeof(tree.DTimestamp{})) + const sizeOfSettingsCacheKey = int(unsafe.Sizeof(SettingsCacheKey{})) + const sizeOfSliceOverHead = int(unsafe.Sizeof([]string{})) + sizeOfSettings := 0 + for _, sEntry := range settingsEntries { + if _, ok := a.settingsCache[sEntry.SettingsCacheKey]; ok { + // Avoid double-counting memory if a key is already in the cache. + continue + } + sizeOfSettings += sizeOfSettingsCacheKey + sizeOfSettings += sizeOfSliceOverHead + sizeOfSettings += len(sEntry.SettingsCacheKey.Username.Normalized()) + for _, s := range sEntry.Settings { + sizeOfSettings += len(s) + } + } + sizeOfEntry := sizeOfUsername + len(username.Normalized()) + + sizeOfAuthInfo + len(aInfo.HashedPassword) + + sizeOfTimestamp + sizeOfSettings + if err := a.boundAccount.Grow(ctx, int64(sizeOfEntry)); err != nil { + // If there is no memory available to cache the entry, we can still + // proceed with authentication so that users are not locked out of + // the database. + log.Ops.Warningf(ctx, "no memory available to cache authentication info: %v", err) + } else { + a.cache[username] = aInfo + for _, sEntry := range settingsEntries { + // Avoid re-storing an existing key. + if _, ok := a.settingsCache[sEntry.SettingsCacheKey]; !ok { + a.settingsCache[sEntry.SettingsCacheKey] = sEntry.Settings + } + } + } + return true + }() +} + +// defaultDatabaseID is used in the settingsCache for entries that should +// apply to all database. +const defaultDatabaseID = 0 + +// defaultUsername is used in the settingsCache for entries that should +// apply to all roles. +var defaultUsername = security.MakeSQLUsernameFromPreNormalizedString("") + +// GenerateSettingsCacheKeys returns a slice of all the SettingsCacheKey +// that are relevant for the given databaseID and username. The slice is +// ordered in descending order of precedence. +func GenerateSettingsCacheKeys( + databaseID descpb.ID, username security.SQLUsername, +) []SettingsCacheKey { + return []SettingsCacheKey{ + { + DatabaseID: databaseID, + Username: username, + }, + { + DatabaseID: defaultDatabaseID, + Username: username, + }, + { + DatabaseID: databaseID, + Username: defaultUsername, + }, + { + DatabaseID: defaultDatabaseID, + Username: defaultUsername, + }, + } } diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index fd4c7e078cfb..5c3e4d1fc990 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -328,7 +328,7 @@ func (p *planner) removeDbRoleSettings(ctx context.Context, dbID descpb.ID) erro if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.DatabaseRoleSettings) { return nil } - _, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.ExecEx( + rowsDeleted, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.ExecEx( ctx, "delete-db-role-settings", p.txn, @@ -337,7 +337,16 @@ func (p *planner) removeDbRoleSettings(ctx context.Context, dbID descpb.ID) erro `DELETE FROM %s WHERE database_id = $1`, authentication.DatabaseRoleSettingsTableName, ), - dbID) + dbID, + ) + if err != nil { + return err + } + if rowsDeleted > 0 && authentication.CacheEnabled.Get(&p.ExecCfg().Settings.SV) { + if err := p.bumpDatabaseRoleSettingsTableVersion(ctx); err != nil { + return err + } + } return err } diff --git a/pkg/sql/drop_role.go b/pkg/sql/drop_role.go index e44b965a42eb..7ea965ae8f0a 100644 --- a/pkg/sql/drop_role.go +++ b/pkg/sql/drop_role.go @@ -258,7 +258,7 @@ func (n *DropRoleNode) startExec(params runParams) error { } // All safe - do the work. - var numRoleMembershipsDeleted int + var numRoleMembershipsDeleted, numRoleSettingsRowsDeleted int for normalizedUsername := range userNames { // Specifically reject special users and roles. Some (root, admin) would fail with // "privileges still exist" first. @@ -308,7 +308,7 @@ func (n *DropRoleNode) startExec(params runParams) error { } // Drop all role memberships involving the user/role. - numRoleMembershipsDeleted, err = params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec( + rowsDeleted, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec( params.ctx, "drop-role-membership", params.p.txn, @@ -318,6 +318,7 @@ func (n *DropRoleNode) startExec(params runParams) error { if err != nil { return err } + numRoleMembershipsDeleted += rowsDeleted _, err = params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec( params.ctx, @@ -335,7 +336,7 @@ func (n *DropRoleNode) startExec(params runParams) error { // TODO(rafi): Remove this condition in 21.2. if params.EvalContext().Settings.Version.IsActive(params.ctx, clusterversion.DatabaseRoleSettings) { - _, err = params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec( + rowsDeleted, err = params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec( params.ctx, opName, params.p.txn, @@ -348,10 +349,11 @@ func (n *DropRoleNode) startExec(params runParams) error { if err != nil { return err } + numRoleSettingsRowsDeleted += rowsDeleted } } - // Bump role-related table versions to force a refresh of membership/password + // Bump role-related table versions to force a refresh of membership/auth // caches. if authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { if err := params.p.bumpUsersTableVersion(params.ctx); err != nil { @@ -360,6 +362,12 @@ func (n *DropRoleNode) startExec(params runParams) error { if err := params.p.bumpRoleOptionsTableVersion(params.ctx); err != nil { return err } + if numRoleSettingsRowsDeleted > 0 && + params.EvalContext().Settings.Version.IsActive(params.ctx, clusterversion.DatabaseRoleSettings) { + if err := params.p.bumpDatabaseRoleSettingsTableVersion(params.ctx); err != nil { + return err + } + } } if numRoleMembershipsDeleted > 0 { if err := params.p.BumpRoleMembershipTableVersion(params.ctx); err != nil { diff --git a/pkg/sql/pgwire/auth.go b/pkg/sql/pgwire/auth.go index 511b4248489d..07f33d83bc8c 100644 --- a/pkg/sql/pgwire/auth.go +++ b/pkg/sql/pgwire/auth.go @@ -15,6 +15,7 @@ import ( "crypto/tls" "fmt" "net" + "strings" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" @@ -88,11 +89,12 @@ func (c *conn) handleAuthentication( // Check that the requested user exists and retrieve the hashed // password in case password authentication is needed. - exists, canLogin, pwRetrievalFn, validUntilFn, err := sql.GetUserHashedPassword( + exists, canLogin, validUntil, defaultSettings, pwRetrievalFn, err := sql.GetUserAuthInfo( ctx, execCfg, authOpt.ie, c.sessionArgs.User, + c.sessionArgs.SessionDefaults["database"], ) if err != nil { log.Warningf(ctx, "user retrieval failed for user=%q: %+v", c.sessionArgs.User, err) @@ -123,7 +125,7 @@ func (c *conn) handleAuthentication( // Ask the method to authenticate. authenticationHook, err := methodFn(ctx, ac, tlsState, pwRetrievalFn, - validUntilFn, execCfg, hbaEntry) + validUntil, execCfg, hbaEntry) if err != nil { ac.LogAuthFailed(ctx, eventpb.AuthFailReason_METHOD_NOT_FOUND, err) @@ -135,6 +137,30 @@ func (c *conn) handleAuthentication( return connClose, sendError(err) } + // Add all the defaults to this session's defaults. If there is an + // error (e.g., a setting that no longer exists, or bad input), + // log a warning instead of preventing login. + // The defaultSettings array is ordered by precedence. This means that if + // if SessionDefaults already has an entry for a given setting name, then + // it should not be replaced. + for _, settingEntry := range defaultSettings { + for _, setting := range settingEntry.Settings { + keyVal := strings.SplitN(setting, "=", 2) + if len(keyVal) != 2 { + log.Ops.Warningf(ctx, "%s has malformed default setting: %q", c.sessionArgs.User, setting) + continue + } + if err := sql.CheckSessionVariableValueValid(ctx, execCfg.Settings, keyVal[0], keyVal[1]); err != nil { + log.Ops.Warningf(ctx, "%s has invalid default setting: %v", c.sessionArgs.User, err) + continue + } + if _, ok := c.sessionArgs.SessionDefaults[keyVal[0]]; !ok { + c.sessionArgs.SessionDefaults[keyVal[0]] = keyVal[1] + } + + } + } + ac.LogAuthOK(ctx) c.msgBuilder.initMsg(pgwirebase.ServerMsgAuth) c.msgBuilder.putInt32(authOK) diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go index f21cd1330a4c..a5c4437ad35e 100644 --- a/pkg/sql/pgwire/auth_methods.go +++ b/pkg/sql/pgwire/auth_methods.go @@ -69,7 +69,7 @@ type AuthMethod func( c AuthConn, tlsState tls.ConnectionState, pwRetrieveFn PasswordRetrievalFn, - pwValidUntilFn PasswordValidUntilFn, + pwValidUntil *tree.DTimestamp, execCfg *sql.ExecutorConfig, entry *hba.Entry, ) (security.UserAuthHook, error) @@ -78,16 +78,12 @@ type AuthMethod func( // password for the user logging in. type PasswordRetrievalFn = func(context.Context) ([]byte, error) -// PasswordValidUntilFn defines a method to retrieve the expiration time -// of the user's password. -type PasswordValidUntilFn = func(context.Context) (*tree.DTimestamp, error) - func authPassword( ctx context.Context, c AuthConn, _ tls.ConnectionState, pwRetrieveFn PasswordRetrievalFn, - pwValidUntilFn PasswordValidUntilFn, + pwValidUntil *tree.DTimestamp, _ *sql.ExecutorConfig, _ *hba.Entry, ) (security.UserAuthHook, error) { @@ -110,12 +106,8 @@ func authPassword( c.LogAuthInfof(ctx, "user has no password defined") } - validUntil, err := pwValidUntilFn(ctx) - if err != nil { - return nil, err - } - if validUntil != nil { - if validUntil.Sub(timeutil.Now()) < 0 { + if pwValidUntil != nil { + if pwValidUntil.Sub(timeutil.Now()) < 0 { c.LogAuthFailed(ctx, eventpb.AuthFailReason_CREDENTIALS_EXPIRED, nil) return nil, errors.New("password is expired") } @@ -139,7 +131,7 @@ func authCert( _ AuthConn, tlsState tls.ConnectionState, _ PasswordRetrievalFn, - _ PasswordValidUntilFn, + _ *tree.DTimestamp, _ *sql.ExecutorConfig, _ *hba.Entry, ) (security.UserAuthHook, error) { @@ -158,7 +150,7 @@ func authCertPassword( c AuthConn, tlsState tls.ConnectionState, pwRetrieveFn PasswordRetrievalFn, - pwValidUntilFn PasswordValidUntilFn, + pwValidUntil *tree.DTimestamp, execCfg *sql.ExecutorConfig, entry *hba.Entry, ) (security.UserAuthHook, error) { @@ -170,7 +162,7 @@ func authCertPassword( c.LogAuthInfof(ctx, "client presented certificate, proceeding with certificate validation") fn = authCert } - return fn(ctx, c, tlsState, pwRetrieveFn, pwValidUntilFn, execCfg, entry) + return fn(ctx, c, tlsState, pwRetrieveFn, pwValidUntil, execCfg, entry) } func authTrust( @@ -178,7 +170,7 @@ func authTrust( _ AuthConn, _ tls.ConnectionState, _ PasswordRetrievalFn, - _ PasswordValidUntilFn, + _ *tree.DTimestamp, _ *sql.ExecutorConfig, _ *hba.Entry, ) (security.UserAuthHook, error) { @@ -190,7 +182,7 @@ func authReject( _ AuthConn, _ tls.ConnectionState, _ PasswordRetrievalFn, - _ PasswordValidUntilFn, + _ *tree.DTimestamp, _ *sql.ExecutorConfig, _ *hba.Entry, ) (security.UserAuthHook, error) { diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index 7b7f4d9cfc13..c14fad1d55ae 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -1592,3 +1592,153 @@ func TestCancelQuery(t *testing.T) { t.Fatalf("unexpected error: %s", err) } } + +func TestRoleDefaultSettings(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + defer db.Close() + + _, err := db.ExecContext(ctx, "CREATE ROLE testuser WITH LOGIN") + require.NoError(t, err) + + pgURL, cleanupFunc := sqlutils.PGUrl( + t, s.ServingSQLAddr(), "TestRoleDefaultSettings" /* prefix */, url.User("testuser"), + ) + defer cleanupFunc() + + for i, tc := range []struct { + setupStmt string + postConnectStmt string + databaseOverride string + searchPathOptOverride string + userOverride string + expectedSearchPath string + }{ + // The test cases need to be in order since the default settings have + // an order of precedence that is being checked here. + { + setupStmt: "ALTER ROLE ALL SET search_path = 'a'", + expectedSearchPath: "a", + }, + { + setupStmt: "ALTER ROLE ALL IN DATABASE defaultdb SET search_path = 'b'", + expectedSearchPath: "b", + }, + { + setupStmt: "ALTER ROLE testuser SET search_path = 'c'", + expectedSearchPath: "c", + }, + { + setupStmt: "ALTER ROLE testuser IN DATABASE defaultdb SET search_path = 'd'", + expectedSearchPath: "d", + }, + { + // Connecting to a different database should use the role-wide default. + databaseOverride: "postgres", + expectedSearchPath: "c", + }, + { + // Connecting to a non-existent database should use the role-wide default + // (and should not error). After connecting, we need to switch to a + // real database so that `SHOW var` works correctly. + databaseOverride: "this_is_not_a_database", + postConnectStmt: "SET DATABASE = defaultdb", + expectedSearchPath: "c", + }, + { + // The setting in the connection URL should take precedence. + searchPathOptOverride: "e", + expectedSearchPath: "e", + }, + { + // Connecting as a different user, should use the database-wide default. + setupStmt: "CREATE ROLE testuser2 WITH LOGIN", + userOverride: "testuser2", + databaseOverride: "defaultdb", + expectedSearchPath: "b", + }, + { + // Connecting as a different user and to a different database should + // use the global default. + userOverride: "testuser2", + databaseOverride: "postgres", + expectedSearchPath: "a", + }, + { + // Test that RESETing the global default works. + setupStmt: "ALTER ROLE ALL RESET search_path", + userOverride: "testuser2", + databaseOverride: "postgres", + expectedSearchPath: `"$user", public`, + }, + { + // Change an existing default setting. + setupStmt: "ALTER ROLE testuser IN DATABASE defaultdb SET search_path = 'f'", + expectedSearchPath: "f", + }, + { + setupStmt: "ALTER ROLE testuser IN DATABASE defaultdb SET search_path = DEFAULT", + expectedSearchPath: "c", + }, + { + setupStmt: "ALTER ROLE testuser SET search_path TO DEFAULT", + expectedSearchPath: "b", + }, + { + // Add a default setting for a different variable. + setupStmt: "ALTER ROLE ALL IN DATABASE defaultdb SET serial_normalization = sql_sequence", + expectedSearchPath: "b", + }, + { + // RESETing the other variable should not affect search_path. + setupStmt: "ALTER ROLE ALL IN DATABASE defaultdb RESET serial_normalization", + expectedSearchPath: "b", + }, + { + // The global default was already reset earlier, so there should be + // no default setting after this. + setupStmt: "ALTER ROLE ALL IN DATABASE defaultdb RESET ALL", + expectedSearchPath: `"$user", public`, + }, + } { + t.Run(fmt.Sprintf("TestRoleDefaultSettings-%d", i), func(t *testing.T) { + _, err := db.ExecContext(ctx, tc.setupStmt) + require.NoError(t, err) + + pgURLCopy := pgURL + if tc.userOverride != "" { + newPGURL, cleanupFunc := sqlutils.PGUrl( + t, s.ServingSQLAddr(), "TestRoleDefaultSettings" /* prefix */, url.User(tc.userOverride), + ) + defer cleanupFunc() + pgURLCopy = newPGURL + } + pgURLCopy.Path = tc.databaseOverride + if tc.searchPathOptOverride != "" { + q := pgURLCopy.Query() + q.Add("search_path", tc.searchPathOptOverride) + pgURLCopy.RawQuery = q.Encode() + } + + thisDB, err := gosql.Open("postgres", pgURLCopy.String()) + require.NoError(t, err) + defer thisDB.Close() + + if tc.postConnectStmt != "" { + _, err = thisDB.ExecContext(ctx, tc.postConnectStmt) + require.NoError(t, err) + } + + var actual string + err = thisDB.QueryRow("SHOW search_path").Scan(&actual) + require.NoError(t, err) + require.Equal(t, tc.expectedSearchPath, actual) + }) + + } + +} diff --git a/pkg/sql/user.go b/pkg/sql/user.go index 130ed41c9281..9c2bb8c1de5a 100644 --- a/pkg/sql/user.go +++ b/pkg/sql/user.go @@ -28,7 +28,7 @@ import ( "github.com/cockroachdb/errors" ) -// GetUserHashedPassword determines if the given user exists and +// GetUserAuthInfo determines if the given user exists and // also returns a password retrieval function. // // The caller is responsible for normalizing the username. @@ -62,57 +62,66 @@ import ( // haven't been any CREATE/ALTER/DROP ROLE commands since, then the cache is // used without a KV lookup. // -func GetUserHashedPassword( - ctx context.Context, execCfg *ExecutorConfig, ie *InternalExecutor, username security.SQLUsername, +func GetUserAuthInfo( + ctx context.Context, + execCfg *ExecutorConfig, + ie *InternalExecutor, + username security.SQLUsername, + databaseName string, ) ( exists bool, canLogin bool, + validUntil *tree.DTimestamp, + defaultSettings []authentication.SettingsCacheEntry, pwRetrieveFn func(ctx context.Context) (hashedPassword []byte, err error), - validUntilFn func(ctx context.Context) (timestamp *tree.DTimestamp, err error), err error, ) { - isRoot := username.IsRootUser() - - if isRoot { + if username.IsRootUser() { // As explained above, for root we report that the user exists // immediately, and delay retrieving the password until strictly // necessary. rootFn := func(ctx context.Context) ([]byte, error) { - authInfo, err := retrieveUserAndPasswordWithCache(ctx, execCfg, ie, isRoot, username) + authInfo, _, err := retrieveAuthInfoWithCache(ctx, execCfg, ie, username, databaseName) return authInfo.HashedPassword, err } // Root user cannot have password expiry and must have login. - validUntilFn := func(ctx context.Context) (*tree.DTimestamp, error) { - return nil, nil - } - return true, true, rootFn, validUntilFn, nil + // It also never has default settings applied to it. + return true, true, nil, nil, rootFn, nil } // Other users must reach for system.users no matter what, because // only that contains the truth about whether the user exists. - authInfo, err := retrieveUserAndPasswordWithCache( - ctx, execCfg, ie, isRoot, username, + authInfo, settingsEntries, err := retrieveAuthInfoWithCache( + ctx, execCfg, ie, username, databaseName, ) - return authInfo.UserExists, authInfo.CanLogin, - func(ctx context.Context) ([]byte, error) { return authInfo.HashedPassword, nil }, - func(ctx context.Context) (*tree.DTimestamp, error) { return authInfo.ValidUntil, nil }, + return authInfo.UserExists, + authInfo.CanLogin, + authInfo.ValidUntil, + settingsEntries, + func(ctx context.Context) ([]byte, error) { + return authInfo.HashedPassword, nil + }, err } -func retrieveUserAndPasswordWithCache( +func retrieveAuthInfoWithCache( ctx context.Context, execCfg *ExecutorConfig, ie *InternalExecutor, - isRoot bool, - normalizedUsername security.SQLUsername, -) (aInfo authentication.AuthInfo, err error) { + username security.SQLUsername, + databaseName string, +) ( + aInfo authentication.AuthInfo, + settingsEntries []authentication.SettingsCacheEntry, + err error, +) { // We may be operating with a timeout. timeout := userLoginTimeout.Get(&ie.s.cfg.Settings.SV) // We don't like long timeouts for root. // (4.5 seconds to not exceed the default 5s timeout configured in many clients.) const maxRootTimeout = 4*time.Second + 500*time.Millisecond - if isRoot && (timeout == 0 || timeout > maxRootTimeout) { + if username.IsRootUser() && (timeout == 0 || timeout > maxRootTimeout) { timeout = maxRootTimeout } @@ -123,33 +132,48 @@ func retrieveUserAndPasswordWithCache( } } err = runFn(func(ctx context.Context) (retErr error) { - aInfo, retErr = execCfg.AuthenticationInfoCache.Get(ctx, execCfg.Settings, ie, execCfg.DB, execCfg.CollectionFactory, normalizedUsername, retrieveUserAndPassword) + aInfo, settingsEntries, retErr = execCfg.AuthenticationInfoCache.Get( + ctx, + execCfg.Settings, + ie, + execCfg.DB, + execCfg.CollectionFactory, + username, + databaseName, + retrieveAuthInfo, + ) return retErr }) if err != nil { // Failed to retrieve the user account. Report in logs for later investigation. - log.Warningf(ctx, "user lookup for %q failed: %v", normalizedUsername, err) + log.Warningf(ctx, "user lookup for %q failed: %v", username, err) err = errors.Wrap(errors.Handled(err), "internal error while retrieving user account") } - return aInfo, err + return aInfo, settingsEntries, err } -func retrieveUserAndPassword( +func retrieveAuthInfo( ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, - normalizedUsername security.SQLUsername, -) (aInfo authentication.AuthInfo, retErr error) { + username security.SQLUsername, + databaseID descpb.ID, + fetchDefaultSettings bool, +) ( + aInfo authentication.AuthInfo, + settingsEntries []authentication.SettingsCacheEntry, + retErr error, +) { // Use fully qualified table name to avoid looking up "".system.users. const getHashedPassword = `SELECT "hashedPassword" FROM system.public.users ` + `WHERE username=$1` values, err := ie.QueryRowEx( ctx, "get-hashed-pwd", txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, - getHashedPassword, normalizedUsername) + getHashedPassword, username) if err != nil { - return authentication.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", normalizedUsername) + return authentication.AuthInfo{}, nil, errors.Wrapf(err, "error looking up user %s", username) } if values != nil { aInfo.UserExists = true @@ -158,33 +182,49 @@ func retrieveUserAndPassword( } } + // Add an empty slice for all the keys so that something gets cached and + // prevents a lookup for the same key from happening later. + keys := authentication.GenerateSettingsCacheKeys(databaseID, username) + settingsEntries = make([]authentication.SettingsCacheEntry, len(keys)) + for i, k := range keys { + settingsEntries[i] = authentication.SettingsCacheEntry{ + SettingsCacheKey: k, + Settings: []string{}, + } + } + if !aInfo.UserExists { - return authentication.AuthInfo{}, nil + return authentication.AuthInfo{}, settingsEntries, nil + } + + // None of the rest of the role options/settings are relevant for root. + if username.IsRootUser() { + return aInfo, settingsEntries, nil } // Use fully qualified table name to avoid looking up "".system.role_options. - getLoginDependencies := `SELECT option, value FROM system.public.role_options ` + + const getLoginDependencies = `SELECT option, value FROM system.public.role_options ` + `WHERE username=$1 AND option IN ('NOLOGIN', 'VALID UNTIL')` - it, err := ie.QueryIteratorEx( + roleOptsIt, err := ie.QueryIteratorEx( ctx, "get-login-dependencies", txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, getLoginDependencies, - normalizedUsername, + username, ) if err != nil { - return authentication.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", normalizedUsername) + return authentication.AuthInfo{}, nil, errors.Wrapf(err, "error looking up user %s", username) } // We have to make sure to close the iterator since we might return from // the for loop early (before Next() returns false). - defer func() { retErr = errors.CombineErrors(retErr, it.Close()) }() + defer func() { retErr = errors.CombineErrors(retErr, roleOptsIt.Close()) }() // To support users created before 20.1, allow all USERS/ROLES to login // if NOLOGIN is not found. aInfo.CanLogin = true var ok bool - for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { - row := it.Cur() + for ok, err = roleOptsIt.Next(ctx); ok; ok, err = roleOptsIt.Next(ctx) { + row := roleOptsIt.Cur() option := string(tree.MustBeDString(row[0])) if option == "NOLOGIN" { @@ -200,14 +240,66 @@ func retrieveUserAndPassword( timeCtx := tree.NewParseTimeContext(timeutil.Now()) aInfo.ValidUntil, _, err = tree.ParseDTimestamp(timeCtx, ts, time.Microsecond) if err != nil { - return authentication.AuthInfo{}, errors.Wrap(err, + return authentication.AuthInfo{}, nil, errors.Wrap(err, "error trying to parse timestamp while retrieving password valid until value") } } } } - return aInfo, err + if !fetchDefaultSettings { + return aInfo, settingsEntries, nil + } + + const getDefaultSettings = ` +SELECT + database_id, role_name, settings +FROM + system.database_role_settings +WHERE + (database_id = 0 AND role_name = $1) + OR (database_id = $2 AND role_name = $1) + OR (database_id = $2 AND role_name = '') + OR (database_id = 0 AND role_name = ''); +` + defaultSettingsIt, err := ie.QueryIteratorEx( + ctx, "get-default-settings", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + getDefaultSettings, + username, + databaseID, + ) + if err != nil { + return authentication.AuthInfo{}, nil, errors.Wrapf(err, "error looking up user %s", username) + } + // We have to make sure to close the iterator since we might return from + // the for loop early (before Next() returns false). + defer func() { retErr = errors.CombineErrors(retErr, defaultSettingsIt.Close()) }() + + for ok, err = defaultSettingsIt.Next(ctx); ok; ok, err = defaultSettingsIt.Next(ctx) { + row := defaultSettingsIt.Cur() + fetechedDatabaseID := descpb.ID(tree.MustBeDOid(row[0]).DInt) + fetchedUsername := security.MakeSQLUsernameFromPreNormalizedString(string(tree.MustBeDString(row[1]))) + settingsDatum := tree.MustBeDArray(row[2]) + fetchedSettings := make([]string, settingsDatum.Len()) + for i, s := range settingsDatum.Array { + fetchedSettings[i] = string(tree.MustBeDString(s)) + } + + thisKey := authentication.SettingsCacheKey{ + DatabaseID: fetechedDatabaseID, + Username: fetchedUsername, + } + // Add the result to the settings list. Note that we don't use a map + // because the list is in order of precedence. + for i, s := range settingsEntries { + if s.SettingsCacheKey == thisKey { + settingsEntries[i].Settings = fetchedSettings + } + } + } + + return aInfo, settingsEntries, err } var userLoginTimeout = settings.RegisterDurationSetting( @@ -298,3 +390,16 @@ func (p *planner) bumpRoleOptionsTableVersion(ctx context.Context) error { ctx, tableDesc, descpb.InvalidMutationID, "updating version for role options table", ) } + +// bumpDatabaseRoleSettingsTableVersion increases the table version for the +// database_role_settings table. +func (p *planner) bumpDatabaseRoleSettingsTableVersion(ctx context.Context) error { + _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, authentication.DatabaseRoleSettingsTableName, true, tree.ResolveAnyTableKind) + if err != nil { + return err + } + + return p.writeSchemaChange( + ctx, tableDesc, descpb.InvalidMutationID, "updating version for database_role_settings table", + ) +} diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index e4f2721f7796..0967e5ce89be 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/delegate" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" @@ -1599,6 +1600,30 @@ func IsSessionVariableConfigurable(varName string) (exists, configurable bool) { return exists, v.Set != nil } +// CheckSessionVariableValueValid returns an error if the value is not valid +// for the given variable. It also returns an error if there is no variable with +// the given name or if the variable is not configurable. +func CheckSessionVariableValueValid( + ctx context.Context, settings *cluster.Settings, varName, varValue string, +) error { + _, sVar, err := getSessionVar(varName, false) + if err != nil { + return err + } + if sVar.Set == nil { + return pgerror.Newf(pgcode.CantChangeRuntimeParam, + "parameter %q cannot be changed", varName) + } + fakeSessionMutator := &sessionDataMutator{ + data: &sessiondata.SessionData{}, + defaults: SessionDefaults(map[string]string{}), + settings: settings, + paramStatusUpdater: &noopParamStatusUpdater{}, + setCurTxnReadOnly: func(bool) {}, + } + return sVar.Set(ctx, fakeSessionMutator, varValue) +} + var varNames []string func getSessionVar(name string, missingOk bool) (bool, sessionVar, error) { From 22a67ef08dac6a18387d0ba5c4a0f9cf9b8dfa32 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Sun, 1 Aug 2021 02:29:17 -0400 Subject: [PATCH 2/8] sql/authentication: move GetDefaultSessionSettings to own function Release note: None --- pkg/sql/authentication/cache.go | 297 ++++++++++++++++++---------- pkg/sql/authentication/constants.go | 13 +- pkg/sql/user.go | 96 +++++---- 3 files changed, 259 insertions(+), 147 deletions(-) diff --git a/pkg/sql/authentication/cache.go b/pkg/sql/authentication/cache.go index aba406393bf7..1517ed91444a 100644 --- a/pkg/sql/authentication/cache.go +++ b/pkg/sql/authentication/cache.go @@ -49,8 +49,8 @@ type AuthInfoCache struct { roleOptionsTableVersion descpb.DescriptorVersion dbRoleSettingsTableVersion descpb.DescriptorVersion boundAccount mon.BoundAccount - // cache is a mapping from username to AuthInfo. - cache map[security.SQLUsername]AuthInfo + // authInfoCache is a mapping from username to AuthInfo. + authInfoCache map[security.SQLUsername]AuthInfo // settingsCache is a mapping from (dbID, username) to default settings. settingsCache map[SettingsCacheKey][]string } @@ -87,35 +87,32 @@ func NewCache(account mon.BoundAccount) *AuthInfoCache { } } -// Get consults the AuthInfoCache and returns the AuthInfo and list of +// GetAuthInfo consults the AuthInfoCache and returns the AuthInfo and list of // SettingsCacheEntry for the provided username and databaseName. If the // information is not in the cache, or if the underlying tables have changed // since the cache was populated, then the readFromStore callback is used to // load new data. -func (a *AuthInfoCache) Get( +func (a *AuthInfoCache) GetAuthInfo( ctx context.Context, settings *cluster.Settings, ie sqlutil.InternalExecutor, db *kv.DB, f *descs.CollectionFactory, username security.SQLUsername, - databaseName string, readFromSystemTables func( ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, username security.SQLUsername, - databaseID descpb.ID, - fetchDefaultSettings bool, - ) (AuthInfo, []SettingsCacheEntry, error), -) (aInfo AuthInfo, settingsEntries []SettingsCacheEntry, err error) { - // TODO(rafi): remove this flag in v21.2. - fetchDefaultSettings := settings.Version.IsActive(ctx, clusterversion.DatabaseRoleSettings) - + ) (AuthInfo, error), +) (aInfo AuthInfo, err error) { + if !CacheEnabled.Get(&settings.SV) { + return readFromSystemTables(ctx, nil /* txn */, ie, username) + } err = f.Txn(ctx, ie, db, func( ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ) (err error) { - var usersTableDesc, roleOptionsTableDesc, dbRoleSettingsTableDesc catalog.TableDescriptor + var usersTableDesc, roleOptionsTableDesc catalog.TableDescriptor _, usersTableDesc, err = descriptors.GetImmutableTableByName( ctx, txn, @@ -134,14 +131,147 @@ func (a *AuthInfoCache) Get( if err != nil { return err } - if fetchDefaultSettings { - _, dbRoleSettingsTableDesc, err = descriptors.GetImmutableTableByName( + if usersTableDesc.IsUncommittedVersion() || + roleOptionsTableDesc.IsUncommittedVersion() || + !CacheEnabled.Get(&settings.SV) { + aInfo, err = readFromSystemTables( ctx, txn, - DatabaseRoleSettingsTableName, - tree.ObjectLookupFlagsWithRequired(), + ie, + username, + ) + if err != nil { + return err + } + } + usersTableVersion := usersTableDesc.GetVersion() + roleOptionsTableVersion := roleOptionsTableDesc.GetVersion() + + // We loop in case the table version changes while looking up + // password or role options. + for { + // Check version and maybe clear cache while holding the mutex. + var found bool + aInfo, found = a.readAuthInfoFromCache(ctx, usersTableVersion, roleOptionsTableVersion, username) + + if found { + return nil + } + + // Lookup the data outside the lock. + aInfo, err = readFromSystemTables( + ctx, + txn, + ie, + username, + ) + if err != nil { + return err + } + + finishedLoop := a.writeAuthInfoBackToCache( + ctx, + &usersTableVersion, + &roleOptionsTableVersion, + aInfo, + username, ) + if finishedLoop { + return nil + } } + }) + return aInfo, err +} + +func (a *AuthInfoCache) readAuthInfoFromCache( + ctx context.Context, + usersTableVersion descpb.DescriptorVersion, + roleOptionsTableVersion descpb.DescriptorVersion, + username security.SQLUsername, +) (AuthInfo, bool) { + a.Lock() + defer a.Unlock() + // We don't need to check dbRoleSettingsTableVersion here, so pass in the + // one we already have. + a.checkStaleness(ctx, usersTableVersion, roleOptionsTableVersion, a.dbRoleSettingsTableVersion) + ai, foundAuthInfo := a.authInfoCache[username] + return ai, foundAuthInfo +} + +// writeAuthInfoBackToCache tries to put the fetched AuthInfo into the +// authInfoCache, and returns true if it succeeded. If the underlying system +// tables have been modified since they were read, the authInfoCache is not +// updated, and the passed in table descriptor versions are updated. +func (a *AuthInfoCache) writeAuthInfoBackToCache( + ctx context.Context, + usersTableVersion *descpb.DescriptorVersion, + roleOptionsTableVersion *descpb.DescriptorVersion, + aInfo AuthInfo, + username security.SQLUsername, +) bool { + a.Lock() + defer a.Unlock() + // Table versions have changed while we were looking: unlock and start over. + if a.usersTableVersion != *usersTableVersion || a.roleOptionsTableVersion != *roleOptionsTableVersion { + *usersTableVersion = a.usersTableVersion + *roleOptionsTableVersion = a.roleOptionsTableVersion + return false + } + // Table version remains the same: update map, unlock, return. + const sizeOfUsername = int(unsafe.Sizeof(security.SQLUsername{})) + const sizeOfAuthInfo = int(unsafe.Sizeof(AuthInfo{})) + const sizeOfTimestamp = int(unsafe.Sizeof(tree.DTimestamp{})) + sizeOfEntry := sizeOfUsername + len(username.Normalized()) + + sizeOfAuthInfo + len(aInfo.HashedPassword) + + sizeOfTimestamp + if err := a.boundAccount.Grow(ctx, int64(sizeOfEntry)); err != nil { + // If there is no memory available to cache the entry, we can still + // proceed with authentication so that users are not locked out of + // the database. + log.Ops.Warningf(ctx, "no memory available to cache authentication info: %v", err) + } else { + a.authInfoCache[username] = aInfo + } + return true +} + +// GetDefaultSettings consults the AuthInfoCache and returns the list of +// SettingsCacheEntry for the provided username and databaseName. If the +// information is not in the cache, or if the underlying tables have changed +// since the cache was populated, then the readFromStore callback is used to +// load new data. +func (a *AuthInfoCache) GetDefaultSettings( + ctx context.Context, + settings *cluster.Settings, + ie sqlutil.InternalExecutor, + db *kv.DB, + f *descs.CollectionFactory, + username security.SQLUsername, + databaseName string, + readFromSystemTables func( + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + username security.SQLUsername, + databaseID descpb.ID, + ) ([]SettingsCacheEntry, error), +) (settingsEntries []SettingsCacheEntry, err error) { + // TODO(rafi): remove this flag in v21.2. + if !settings.Version.IsActive(ctx, clusterversion.DatabaseRoleSettings) { + return nil, nil + } + + err = f.Txn(ctx, ie, db, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + ) (err error) { + var dbRoleSettingsTableDesc catalog.TableDescriptor + _, dbRoleSettingsTableDesc, err = descriptors.GetImmutableTableByName( + ctx, + txn, + DatabaseRoleSettingsTableName, + tree.ObjectLookupFlagsWithRequired(), + ) if err != nil { return err } @@ -158,82 +288,67 @@ func (a *AuthInfoCache) Get( } } - if usersTableDesc.IsUncommittedVersion() || - roleOptionsTableDesc.IsUncommittedVersion() || - (fetchDefaultSettings && dbRoleSettingsTableDesc.IsUncommittedVersion()) || - !CacheEnabled.Get(&settings.SV) { - aInfo, settingsEntries, err = readFromSystemTables( + if dbRoleSettingsTableDesc.IsUncommittedVersion() || !CacheEnabled.Get(&settings.SV) { + settingsEntries, err = readFromSystemTables( ctx, txn, ie, username, databaseID, - fetchDefaultSettings, ) - if err != nil { + if err != nil || !CacheEnabled.Get(&settings.SV) { return err } } - usersTableVersion := usersTableDesc.GetVersion() - roleOptionsTableVersion := roleOptionsTableDesc.GetVersion() - dbRoleSettingsTableVersion := descpb.DescriptorVersion(0) - if fetchDefaultSettings { - dbRoleSettingsTableVersion = dbRoleSettingsTableDesc.GetVersion() - } + dbRoleSettingsTableVersion := dbRoleSettingsTableDesc.GetVersion() // We loop in case the table version changes while looking up // password or role options. for { // Check version and maybe clear cache while holding the mutex. var found bool - aInfo, settingsEntries, found = a.readFromCache(ctx, usersTableVersion, roleOptionsTableVersion, dbRoleSettingsTableVersion, username, databaseID) + settingsEntries, found = a.readDefaultSettingsFromCache(ctx, dbRoleSettingsTableVersion, username, databaseID) if found { return nil } // Lookup the data outside the lock. - aInfo, settingsEntries, err = readFromSystemTables( + settingsEntries, err = readFromSystemTables( ctx, txn, ie, username, databaseID, - fetchDefaultSettings, ) if err != nil { return err } - finishedLoop := a.writeBackToCache( + finishedLoop := a.writeDefaultSettingsBackToCache( ctx, - &usersTableVersion, - &roleOptionsTableVersion, &dbRoleSettingsTableVersion, - aInfo, settingsEntries, - username, ) if finishedLoop { return nil } } }) - return aInfo, settingsEntries, err + return settingsEntries, err } -func (a *AuthInfoCache) readFromCache( +func (a *AuthInfoCache) readDefaultSettingsFromCache( ctx context.Context, - usersTableVersion descpb.DescriptorVersion, - roleOptionsTableVersion descpb.DescriptorVersion, dbRoleSettingsTableVersion descpb.DescriptorVersion, username security.SQLUsername, databaseID descpb.ID, -) (AuthInfo, []SettingsCacheEntry, bool) { +) ([]SettingsCacheEntry, bool) { a.Lock() defer a.Unlock() - a.checkStaleness(ctx, usersTableVersion, roleOptionsTableVersion, dbRoleSettingsTableVersion) - ai, foundAuthInfo := a.cache[username] + // We don't need to check usersTableVersion or roleOptionsTableVersion here, \ + // so pass in the values we already have. + a.checkStaleness(ctx, a.usersTableVersion, a.roleOptionsTableVersion, dbRoleSettingsTableVersion) foundAllDefaultSettings := true var sEntries []SettingsCacheEntry // Search through the cache for the settings entries we need. Note @@ -246,93 +361,48 @@ func (a *AuthInfoCache) readFromCache( } sEntries = append(sEntries, SettingsCacheEntry{k, s}) } - return ai, sEntries, foundAuthInfo && foundAllDefaultSettings + return sEntries, foundAllDefaultSettings } -func (a *AuthInfoCache) checkStaleness( +// writeDefaultSettingsBackToCache tries to put the fetched SettingsCacheentry +// list into the settingsCache, and returns true if it succeeded. If the +// underlying system tables have been modified since they were read, the +// settingsCache is not updated, and the passed in table descriptor version is +// updated. +func (a *AuthInfoCache) writeDefaultSettingsBackToCache( ctx context.Context, - usersTableVersion descpb.DescriptorVersion, - roleOptionsTableVersion descpb.DescriptorVersion, - dbRoleSettingsTableVersion descpb.DescriptorVersion, -) { - if a.usersTableVersion != usersTableVersion { - // Update users table version and drop the map. - a.usersTableVersion = usersTableVersion - a.cache = make(map[security.SQLUsername]AuthInfo) - a.settingsCache = make(map[SettingsCacheKey][]string) - a.boundAccount.Empty(ctx) - } - if a.roleOptionsTableVersion != roleOptionsTableVersion { - // Update role_options table version and drop the map. - a.roleOptionsTableVersion = roleOptionsTableVersion - a.cache = make(map[security.SQLUsername]AuthInfo) - a.settingsCache = make(map[SettingsCacheKey][]string) - a.boundAccount.Empty(ctx) - } - if a.dbRoleSettingsTableVersion != dbRoleSettingsTableVersion { - // Update database_role_settings table version and drop the map. - a.dbRoleSettingsTableVersion = dbRoleSettingsTableVersion - a.cache = make(map[security.SQLUsername]AuthInfo) - a.settingsCache = make(map[SettingsCacheKey][]string) - a.boundAccount.Empty(ctx) - } -} - -// writeBackToCache tries to put the fetched data into the cache, and returns -// true if it succeeded. If the underlying system tables have been modified -// since they were read, the cache is not updated, and the passed in table -// descriptor versions are all updated. -func (a *AuthInfoCache) writeBackToCache( - ctx context.Context, - usersTableVersion *descpb.DescriptorVersion, - roleOptionsTableVersion *descpb.DescriptorVersion, dbRoleSettingsTableVersion *descpb.DescriptorVersion, - aInfo AuthInfo, settingsEntries []SettingsCacheEntry, - username security.SQLUsername, ) bool { return func() bool { a.Lock() defer a.Unlock() // Table version has changed while we were looking: unlock and start over. - if a.usersTableVersion != *usersTableVersion || - a.roleOptionsTableVersion != *roleOptionsTableVersion || - a.dbRoleSettingsTableVersion != *dbRoleSettingsTableVersion { - *usersTableVersion = a.usersTableVersion - *roleOptionsTableVersion = a.roleOptionsTableVersion + if a.dbRoleSettingsTableVersion != *dbRoleSettingsTableVersion { *dbRoleSettingsTableVersion = a.dbRoleSettingsTableVersion return false } // Table version remains the same: update map, unlock, return. - const sizeOfUsername = int(unsafe.Sizeof(security.SQLUsername{})) - const sizeOfAuthInfo = int(unsafe.Sizeof(AuthInfo{})) - const sizeOfTimestamp = int(unsafe.Sizeof(tree.DTimestamp{})) - const sizeOfSettingsCacheKey = int(unsafe.Sizeof(SettingsCacheKey{})) - const sizeOfSliceOverHead = int(unsafe.Sizeof([]string{})) + const sizeOfSettingsCacheEntry = int(unsafe.Sizeof(SettingsCacheEntry{})) sizeOfSettings := 0 for _, sEntry := range settingsEntries { if _, ok := a.settingsCache[sEntry.SettingsCacheKey]; ok { // Avoid double-counting memory if a key is already in the cache. continue } - sizeOfSettings += sizeOfSettingsCacheKey - sizeOfSettings += sizeOfSliceOverHead + sizeOfSettings += sizeOfSettingsCacheEntry sizeOfSettings += len(sEntry.SettingsCacheKey.Username.Normalized()) for _, s := range sEntry.Settings { sizeOfSettings += len(s) } } - sizeOfEntry := sizeOfUsername + len(username.Normalized()) + - sizeOfAuthInfo + len(aInfo.HashedPassword) + - sizeOfTimestamp + sizeOfSettings - if err := a.boundAccount.Grow(ctx, int64(sizeOfEntry)); err != nil { + if err := a.boundAccount.Grow(ctx, int64(sizeOfSettings)); err != nil { // If there is no memory available to cache the entry, we can still // proceed with authentication so that users are not locked out of // the database. log.Ops.Warningf(ctx, "no memory available to cache authentication info: %v", err) } else { - a.cache[username] = aInfo for _, sEntry := range settingsEntries { // Avoid re-storing an existing key. if _, ok := a.settingsCache[sEntry.SettingsCacheKey]; !ok { @@ -344,13 +414,24 @@ func (a *AuthInfoCache) writeBackToCache( }() } -// defaultDatabaseID is used in the settingsCache for entries that should -// apply to all database. -const defaultDatabaseID = 0 - -// defaultUsername is used in the settingsCache for entries that should -// apply to all roles. -var defaultUsername = security.MakeSQLUsernameFromPreNormalizedString("") +func (a *AuthInfoCache) checkStaleness( + ctx context.Context, + usersTableVersion descpb.DescriptorVersion, + roleOptionsTableVersion descpb.DescriptorVersion, + dbRoleSettingsTableVersion descpb.DescriptorVersion, +) { + if a.usersTableVersion != usersTableVersion || + // Update table versions and drop the maps. + a.roleOptionsTableVersion != roleOptionsTableVersion || + a.dbRoleSettingsTableVersion != dbRoleSettingsTableVersion { + a.usersTableVersion = usersTableVersion + a.roleOptionsTableVersion = roleOptionsTableVersion + a.dbRoleSettingsTableVersion = dbRoleSettingsTableVersion + a.authInfoCache = make(map[security.SQLUsername]AuthInfo) + a.settingsCache = make(map[SettingsCacheKey][]string) + a.boundAccount.Empty(ctx) + } +} // GenerateSettingsCacheKeys returns a slice of all the SettingsCacheKey // that are relevant for the given databaseID and username. The slice is diff --git a/pkg/sql/authentication/constants.go b/pkg/sql/authentication/constants.go index fcb656abc169..73de3043df1b 100644 --- a/pkg/sql/authentication/constants.go +++ b/pkg/sql/authentication/constants.go @@ -10,7 +10,10 @@ package authentication -import "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +import ( + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) // UsersTableName represents system.users. var UsersTableName = tree.NewTableNameWithSchema("system", tree.PublicSchemaName, "users") @@ -20,3 +23,11 @@ var RoleOptionsTableName = tree.NewTableNameWithSchema("system", tree.PublicSche // DatabaseRoleSettingsTableName represents system.database_role_settings. var DatabaseRoleSettingsTableName = tree.NewTableNameWithSchema("system", tree.PublicSchemaName, "database_role_settings") + +// defaultDatabaseID is used in the settingsCache for entries that should +// apply to all database. +const defaultDatabaseID = 0 + +// defaultUsername is used in the settingsCache for entries that should +// apply to all roles. +var defaultUsername = security.MakeSQLUsernameFromPreNormalizedString("") diff --git a/pkg/sql/user.go b/pkg/sql/user.go index 9c2bb8c1de5a..e93cb5cb0bc1 100644 --- a/pkg/sql/user.go +++ b/pkg/sql/user.go @@ -81,7 +81,7 @@ func GetUserAuthInfo( // immediately, and delay retrieving the password until strictly // necessary. rootFn := func(ctx context.Context) ([]byte, error) { - authInfo, _, err := retrieveAuthInfoWithCache(ctx, execCfg, ie, username, databaseName) + authInfo, _, err := retrieveSessionInitInfoWithCache(ctx, execCfg, ie, username, databaseName) return authInfo.HashedPassword, err } @@ -92,7 +92,7 @@ func GetUserAuthInfo( // Other users must reach for system.users no matter what, because // only that contains the truth about whether the user exists. - authInfo, settingsEntries, err := retrieveAuthInfoWithCache( + authInfo, settingsEntries, err := retrieveSessionInitInfoWithCache( ctx, execCfg, ie, username, databaseName, ) return authInfo.UserExists, @@ -105,7 +105,7 @@ func GetUserAuthInfo( err } -func retrieveAuthInfoWithCache( +func retrieveSessionInitInfoWithCache( ctx context.Context, execCfg *ExecutorConfig, ie *InternalExecutor, @@ -132,16 +132,32 @@ func retrieveAuthInfoWithCache( } } err = runFn(func(ctx context.Context) (retErr error) { - aInfo, settingsEntries, retErr = execCfg.AuthenticationInfoCache.Get( + aInfo, retErr = execCfg.AuthenticationInfoCache.GetAuthInfo( ctx, execCfg.Settings, ie, execCfg.DB, execCfg.CollectionFactory, username, - databaseName, retrieveAuthInfo, ) + if retErr != nil { + return retErr + } + // Avoid lookingup default settings for root and non-existent users. + if username.IsRootUser() || !aInfo.UserExists { + return nil + } + settingsEntries, retErr = execCfg.AuthenticationInfoCache.GetDefaultSettings( + ctx, + execCfg.Settings, + ie, + execCfg.DB, + execCfg.CollectionFactory, + username, + databaseName, + retrieveDefaultSettings, + ) return retErr }) @@ -154,17 +170,8 @@ func retrieveAuthInfoWithCache( } func retrieveAuthInfo( - ctx context.Context, - txn *kv.Txn, - ie sqlutil.InternalExecutor, - username security.SQLUsername, - databaseID descpb.ID, - fetchDefaultSettings bool, -) ( - aInfo authentication.AuthInfo, - settingsEntries []authentication.SettingsCacheEntry, - retErr error, -) { + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, username security.SQLUsername, +) (aInfo authentication.AuthInfo, retErr error) { // Use fully qualified table name to avoid looking up "".system.users. const getHashedPassword = `SELECT "hashedPassword" FROM system.public.users ` + `WHERE username=$1` @@ -173,7 +180,7 @@ func retrieveAuthInfo( sessiondata.InternalExecutorOverride{User: security.RootUserName()}, getHashedPassword, username) if err != nil { - return authentication.AuthInfo{}, nil, errors.Wrapf(err, "error looking up user %s", username) + return authentication.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", username) } if values != nil { aInfo.UserExists = true @@ -182,24 +189,13 @@ func retrieveAuthInfo( } } - // Add an empty slice for all the keys so that something gets cached and - // prevents a lookup for the same key from happening later. - keys := authentication.GenerateSettingsCacheKeys(databaseID, username) - settingsEntries = make([]authentication.SettingsCacheEntry, len(keys)) - for i, k := range keys { - settingsEntries[i] = authentication.SettingsCacheEntry{ - SettingsCacheKey: k, - Settings: []string{}, - } - } - if !aInfo.UserExists { - return authentication.AuthInfo{}, settingsEntries, nil + return authentication.AuthInfo{}, nil } - // None of the rest of the role options/settings are relevant for root. + // None of the rest of the role options are relevant for root. if username.IsRootUser() { - return aInfo, settingsEntries, nil + return aInfo, nil } // Use fully qualified table name to avoid looking up "".system.role_options. @@ -213,7 +209,7 @@ func retrieveAuthInfo( username, ) if err != nil { - return authentication.AuthInfo{}, nil, errors.Wrapf(err, "error looking up user %s", username) + return authentication.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", username) } // We have to make sure to close the iterator since we might return from // the for loop early (before Next() returns false). @@ -240,22 +236,45 @@ func retrieveAuthInfo( timeCtx := tree.NewParseTimeContext(timeutil.Now()) aInfo.ValidUntil, _, err = tree.ParseDTimestamp(timeCtx, ts, time.Microsecond) if err != nil { - return authentication.AuthInfo{}, nil, errors.Wrap(err, + return authentication.AuthInfo{}, errors.Wrap(err, "error trying to parse timestamp while retrieving password valid until value") } } } } - if !fetchDefaultSettings { - return aInfo, settingsEntries, nil + return aInfo, err +} + +func retrieveDefaultSettings( + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + username security.SQLUsername, + databaseID descpb.ID, +) (settingsEntries []authentication.SettingsCacheEntry, retErr error) { + // Add an empty slice for all the keys so that something gets cached and + // prevents a lookup for the same key from happening later. + keys := authentication.GenerateSettingsCacheKeys(databaseID, username) + settingsEntries = make([]authentication.SettingsCacheEntry, len(keys)) + for i, k := range keys { + settingsEntries[i] = authentication.SettingsCacheEntry{ + SettingsCacheKey: k, + Settings: []string{}, + } + } + + // The default settings are not relevant for root. + if username.IsRootUser() { + return settingsEntries, nil } + // Use fully qualified table name to avoid looking up "".system.role_options. const getDefaultSettings = ` SELECT database_id, role_name, settings FROM - system.database_role_settings + system.public.database_role_settings WHERE (database_id = 0 AND role_name = $1) OR (database_id = $2 AND role_name = $1) @@ -270,12 +289,13 @@ WHERE databaseID, ) if err != nil { - return authentication.AuthInfo{}, nil, errors.Wrapf(err, "error looking up user %s", username) + return nil, errors.Wrapf(err, "error looking up user %s", username) } // We have to make sure to close the iterator since we might return from // the for loop early (before Next() returns false). defer func() { retErr = errors.CombineErrors(retErr, defaultSettingsIt.Close()) }() + var ok bool for ok, err = defaultSettingsIt.Next(ctx); ok; ok, err = defaultSettingsIt.Next(ctx) { row := defaultSettingsIt.Cur() fetechedDatabaseID := descpb.ID(tree.MustBeDOid(row[0]).DInt) @@ -299,7 +319,7 @@ WHERE } } - return aInfo, settingsEntries, err + return settingsEntries, err } var userLoginTimeout = settings.RegisterDurationSetting( From 617e51e7de0750a20ccde45f2c7959670aa650c1 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Sun, 1 Aug 2021 02:35:15 -0400 Subject: [PATCH 3/8] sql: rename sql/authentication to sql/sessioninit Add a few comments to increase readability. Release note: None --- pkg/ccl/serverccl/role_authentication_test.go | 2 +- pkg/server/BUILD.bazel | 2 +- pkg/server/authentication.go | 4 +- pkg/server/server_sql.go | 4 +- pkg/sql/BUILD.bazel | 2 +- pkg/sql/alter_role.go | 18 ++--- pkg/sql/authorization.go | 4 +- pkg/sql/create_role.go | 8 +- pkg/sql/drop_database.go | 6 +- pkg/sql/drop_role.go | 8 +- pkg/sql/exec_util.go | 7 +- pkg/sql/pgwire/auth.go | 4 +- .../BUILD.bazel | 25 ++++++ .../{authentication => sessioninit}/cache.go | 80 ++++++++++--------- .../constants.go | 2 +- pkg/sql/set_cluster_setting.go | 4 +- pkg/sql/user.go | 67 ++++++++-------- 17 files changed, 139 insertions(+), 108 deletions(-) rename pkg/sql/{authentication => sessioninit}/BUILD.bazel (52%) rename pkg/sql/{authentication => sessioninit}/cache.go (85%) rename pkg/sql/{authentication => sessioninit}/constants.go (98%) diff --git a/pkg/ccl/serverccl/role_authentication_test.go b/pkg/ccl/serverccl/role_authentication_test.go index 649a7b116b30..65e757844125 100644 --- a/pkg/ccl/serverccl/role_authentication_test.go +++ b/pkg/ccl/serverccl/role_authentication_test.go @@ -115,7 +115,7 @@ func TestVerifyPassword(t *testing.T) { t.Run("", func(t *testing.T) { execCfg := s.ExecutorConfig().(sql.ExecutorConfig) username := security.MakeSQLUsernameFromPreNormalizedString(tc.username) - exists, canLogin, validUntil, _, pwRetrieveFn, err := sql.GetUserAuthInfo( + exists, canLogin, validUntil, _, pwRetrieveFn, err := sql.GetUserSessionInitInfo( context.Background(), &execCfg, &ie, username, "", /* databaseName */ ) diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index dc18dc949792..7f7929307225 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -111,7 +111,6 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", - "//pkg/sql/authentication", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catconstants", @@ -141,6 +140,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", + "//pkg/sql/sessioninit", "//pkg/sql/sqlinstance", "//pkg/sql/sqlinstance/instanceprovider", "//pkg/sql/sqlliveness", diff --git a/pkg/server/authentication.go b/pkg/server/authentication.go index b06ec9bf29a0..3a5575d9a8e3 100644 --- a/pkg/server/authentication.go +++ b/pkg/server/authentication.go @@ -254,7 +254,7 @@ func (s *authenticationServer) UserLoginFromSSO( // without further normalization. username, _ := security.MakeSQLUsernameFromUserInput(reqUsername, security.UsernameValidation) - exists, canLogin, _, _, _, err := sql.GetUserAuthInfo( + exists, canLogin, _, _, _, err := sql.GetUserSessionInitInfo( ctx, s.server.sqlServer.execCfg, s.server.sqlServer.execCfg.InternalExecutor, @@ -417,7 +417,7 @@ WHERE id = $1` func (s *authenticationServer) verifyPassword( ctx context.Context, username security.SQLUsername, password string, ) (valid bool, expired bool, err error) { - exists, canLogin, validUntil, _, pwRetrieveFn, err := sql.GetUserAuthInfo( + exists, canLogin, validUntil, _, pwRetrieveFn, err := sql.GetUserSessionInitInfo( ctx, s.server.sqlServer.execCfg, s.server.sqlServer.execCfg.InternalExecutor, diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 718b8e6e103a..278b6c8be8bd 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -50,7 +50,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/server/tracedumper" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables" @@ -71,6 +70,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance/instanceprovider" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" @@ -589,7 +589,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { HistogramWindowInterval: cfg.HistogramWindowInterval(), RangeDescriptorCache: cfg.distSender.RangeDescriptorCache(), RoleMemberCache: sql.NewMembershipCache(serverCacheMemoryMonitor.MakeBoundAccount()), - AuthenticationInfoCache: authentication.NewCache(serverCacheMemoryMonitor.MakeBoundAccount()), + SessionInitCache: sessioninit.NewCache(serverCacheMemoryMonitor.MakeBoundAccount()), RootMemoryMonitor: rootSQLMemoryMonitor, TestingKnobs: sqlExecutorTestingKnobs, CompactEngineSpanFunc: compactEngineSpanFunc, diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index da7cdb6573f3..500b009d2be8 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -274,7 +274,6 @@ go_library( "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", - "//pkg/sql/authentication", "//pkg/sql/backfill", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", @@ -348,6 +347,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", + "//pkg/sql/sessioninit", "//pkg/sql/sessionphase", "//pkg/sql/span", "//pkg/sql/sqlerrors", diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go index d657cdce51a9..4b8553016a4f 100644 --- a/pkg/sql/alter_role.go +++ b/pkg/sql/alter_role.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -25,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" @@ -178,7 +178,7 @@ func (n *alterRoleNode) startExec(params runParams) error { opName, params.p.txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, - fmt.Sprintf("SELECT 1 FROM %s WHERE username = $1", authentication.UsersTableName), + fmt.Sprintf("SELECT 1 FROM %s WHERE username = $1", sessioninit.UsersTableName), normalizedUsername, ) if err != nil { @@ -237,7 +237,7 @@ func (n *alterRoleNode) startExec(params runParams) error { if err != nil { return err } - if authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { + if sessioninit.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { // Bump user table versions to force a refresh of AuthInfo cache. if err := params.p.bumpUsersTableVersion(params.ctx); err != nil { return err @@ -287,7 +287,7 @@ func (n *alterRoleNode) startExec(params runParams) error { optStrs[i] = n.roleOptions[i].String() } - if authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { + if sessioninit.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { // Bump role_options table versions to force a refresh of AuthInfo cache. if err := params.p.bumpRoleOptionsTableVersion(params.ctx); err != nil { return err @@ -456,11 +456,11 @@ func (n *alterRoleSetNode) startExec(params runParams) error { var deleteQuery = fmt.Sprintf( `DELETE FROM %s WHERE database_id = $1 AND role_name = $2`, - authentication.DatabaseRoleSettingsTableName, + sessioninit.DatabaseRoleSettingsTableName, ) var upsertQuery = fmt.Sprintf( `UPSERT INTO %s (database_id, role_name, settings) VALUES ($1, $2, $3)`, - authentication.DatabaseRoleSettingsTableName, + sessioninit.DatabaseRoleSettingsTableName, ) // Instead of inserting an empty settings array, this function will make @@ -494,7 +494,7 @@ func (n *alterRoleSetNode) startExec(params runParams) error { return internalExecErr } - if rowsAffected > 0 && authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { + if rowsAffected > 0 && sessioninit.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { // Bump database_role_settings table versions to force a refresh of AuthInfo cache. if err := params.p.bumpDatabaseRoleSettingsTableVersion(params.ctx); err != nil { return err @@ -569,7 +569,7 @@ func (n *alterRoleSetNode) getRoleName( opName, params.p.txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, - fmt.Sprintf("SELECT 1 FROM %s WHERE username = $1", authentication.UsersTableName), + fmt.Sprintf("SELECT 1 FROM %s WHERE username = $1", sessioninit.UsersTableName), roleName, ) if err != nil { @@ -600,7 +600,7 @@ func (n *alterRoleSetNode) makeNewSettings( ) (hasOldSettings bool, newSettings []string, err error) { var selectQuery = fmt.Sprintf( `SELECT settings FROM %s WHERE database_id = $1 AND role_name = $2`, - authentication.DatabaseRoleSettingsTableName, + sessioninit.DatabaseRoleSettingsTableName, ) datums, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryRowEx( params.ctx, diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go index f21c364384b3..fa540f50bdcb 100644 --- a/pkg/sql/authorization.go +++ b/pkg/sql/authorization.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -31,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -474,7 +474,7 @@ func (p *planner) HasRoleOption(ctx context.Context, roleOption roleoption.Optio sessiondata.InternalExecutorOverride{User: security.RootUserName()}, fmt.Sprintf( `SELECT 1 from %s WHERE option = '%s' AND username = $1 LIMIT 1`, - authentication.RoleOptionsTableName, roleOption.String()), user.Normalized()) + sessioninit.RoleOptionsTableName, roleOption.String()), user.Normalized()) if err != nil { return false, err } diff --git a/pkg/sql/create_role.go b/pkg/sql/create_role.go index fd956474e8d7..abe3fdc32922 100644 --- a/pkg/sql/create_role.go +++ b/pkg/sql/create_role.go @@ -16,12 +16,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" @@ -155,7 +155,7 @@ func (n *CreateRoleNode) startExec(params runParams) error { opName, params.p.txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, - fmt.Sprintf(`select "isRole" from %s where username = $1`, authentication.UsersTableName), + fmt.Sprintf(`select "isRole" from %s where username = $1`, sessioninit.UsersTableName), normalizedUsername, ) if err != nil { @@ -174,7 +174,7 @@ func (n *CreateRoleNode) startExec(params runParams) error { params.ctx, opName, params.p.txn, - fmt.Sprintf("insert into %s values ($1, $2, $3)", authentication.UsersTableName), + fmt.Sprintf("insert into %s values ($1, $2, $3)", sessioninit.UsersTableName), normalizedUsername, hashedPassword, n.isRole, @@ -225,7 +225,7 @@ func (n *CreateRoleNode) startExec(params runParams) error { } } - if authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { + if sessioninit.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { // Bump role-related table versions to force a refresh of AuthInfo cache. if err := params.p.bumpUsersTableVersion(params.ctx); err != nil { return err diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 5c3e4d1fc990..3cb44f448054 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -31,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" @@ -335,14 +335,14 @@ func (p *planner) removeDbRoleSettings(ctx context.Context, dbID descpb.ID) erro sessiondata.InternalExecutorOverride{User: security.RootUserName()}, fmt.Sprintf( `DELETE FROM %s WHERE database_id = $1`, - authentication.DatabaseRoleSettingsTableName, + sessioninit.DatabaseRoleSettingsTableName, ), dbID, ) if err != nil { return err } - if rowsDeleted > 0 && authentication.CacheEnabled.Get(&p.ExecCfg().Settings.SV) { + if rowsDeleted > 0 && sessioninit.CacheEnabled.Get(&p.ExecCfg().Settings.SV) { if err := p.bumpDatabaseRoleSettingsTableVersion(ctx); err != nil { return err } diff --git a/pkg/sql/drop_role.go b/pkg/sql/drop_role.go index 7ea965ae8f0a..895503eff959 100644 --- a/pkg/sql/drop_role.go +++ b/pkg/sql/drop_role.go @@ -17,12 +17,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" @@ -326,7 +326,7 @@ func (n *DropRoleNode) startExec(params runParams) error { params.p.txn, fmt.Sprintf( `DELETE FROM %s WHERE username=$1`, - authentication.RoleOptionsTableName, + sessioninit.RoleOptionsTableName, ), normalizedUsername, ) @@ -342,7 +342,7 @@ func (n *DropRoleNode) startExec(params runParams) error { params.p.txn, fmt.Sprintf( `DELETE FROM %s WHERE role_name = $1`, - authentication.DatabaseRoleSettingsTableName, + sessioninit.DatabaseRoleSettingsTableName, ), normalizedUsername, ) @@ -355,7 +355,7 @@ func (n *DropRoleNode) startExec(params runParams) error { // Bump role-related table versions to force a refresh of membership/auth // caches. - if authentication.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { + if sessioninit.CacheEnabled.Get(¶ms.p.ExecCfg().Settings.SV) { if err := params.p.bumpUsersTableVersion(params.ctx); err != nil { return err } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index b37cec975686..1897ff10b69a 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -53,7 +53,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -76,6 +75,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics" @@ -942,8 +942,9 @@ type ExecutorConfig struct { // Role membership cache. RoleMemberCache *MembershipCache - // AuthInfo cache. - AuthenticationInfoCache *authentication.AuthInfoCache + // SessionInitCache cache; contains information used during authentication + // and per-role default settings. + SessionInitCache *sessioninit.Cache // ProtectedTimestampProvider encapsulates the protected timestamp subsystem. ProtectedTimestampProvider protectedts.Provider diff --git a/pkg/sql/pgwire/auth.go b/pkg/sql/pgwire/auth.go index 07f33d83bc8c..cb27a7bab9d5 100644 --- a/pkg/sql/pgwire/auth.go +++ b/pkg/sql/pgwire/auth.go @@ -89,7 +89,7 @@ func (c *conn) handleAuthentication( // Check that the requested user exists and retrieve the hashed // password in case password authentication is needed. - exists, canLogin, validUntil, defaultSettings, pwRetrievalFn, err := sql.GetUserAuthInfo( + exists, canLogin, validUntil, defaultSettings, pwRetrievalFn, err := sql.GetUserSessionInitInfo( ctx, execCfg, authOpt.ie, @@ -141,7 +141,7 @@ func (c *conn) handleAuthentication( // error (e.g., a setting that no longer exists, or bad input), // log a warning instead of preventing login. // The defaultSettings array is ordered by precedence. This means that if - // if SessionDefaults already has an entry for a given setting name, then + // SessionDefaults already has an entry for a given setting name, then // it should not be replaced. for _, settingEntry := range defaultSettings { for _, setting := range settingEntry.Settings { diff --git a/pkg/sql/authentication/BUILD.bazel b/pkg/sql/sessioninit/BUILD.bazel similarity index 52% rename from pkg/sql/authentication/BUILD.bazel rename to pkg/sql/sessioninit/BUILD.bazel index d686684115f8..8993c162d271 100644 --- a/pkg/sql/authentication/BUILD.bazel +++ b/pkg/sql/sessioninit/BUILD.bazel @@ -24,3 +24,28 @@ go_library( "//pkg/util/syncutil", ], ) + +go_library( + name = "sessioninit", + srcs = [ + "cache.go", + "constants.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/sessioninit", + visibility = ["//visibility:public"], + deps = [ + "//pkg/clusterversion", + "//pkg/kv", + "//pkg/security", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/descs", + "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", + "//pkg/util/log", + "//pkg/util/mon", + "//pkg/util/syncutil", + ], +) diff --git a/pkg/sql/authentication/cache.go b/pkg/sql/sessioninit/cache.go similarity index 85% rename from pkg/sql/authentication/cache.go rename to pkg/sql/sessioninit/cache.go index 1517ed91444a..24fc5c0f6f95 100644 --- a/pkg/sql/authentication/cache.go +++ b/pkg/sql/sessioninit/cache.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package authentication +package sessioninit import ( "context" @@ -33,7 +33,7 @@ import ( var CacheEnabledSettingName = "server.authentication_cache.enabled" // CacheEnabled is a cluster setting that determines if the -// AuthInfoCache and associated logic is enabled. +// sessioninit.Cache and associated logic is enabled. var CacheEnabled = settings.RegisterBoolSetting( CacheEnabledSettingName, "enables a cache used during authentication to avoid lookups to system tables "+ @@ -41,9 +41,9 @@ var CacheEnabled = settings.RegisterBoolSetting( true, ).WithPublic() -// AuthInfoCache is a shared cache for hashed passwords and other -// information used during user authentication and session initialization. -type AuthInfoCache struct { +// Cache is a shared cache for hashed passwords and other information used +// during user authentication and session initialization. +type Cache struct { syncutil.Mutex usersTableVersion descpb.DescriptorVersion roleOptionsTableVersion descpb.DescriptorVersion @@ -80,19 +80,18 @@ type SettingsCacheEntry struct { Settings []string } -// NewCache initializes a new AuthInfoCache. -func NewCache(account mon.BoundAccount) *AuthInfoCache { - return &AuthInfoCache{ +// NewCache initializes a new sessioninit.Cache. +func NewCache(account mon.BoundAccount) *Cache { + return &Cache{ boundAccount: account, } } -// GetAuthInfo consults the AuthInfoCache and returns the AuthInfo and list of -// SettingsCacheEntry for the provided username and databaseName. If the -// information is not in the cache, or if the underlying tables have changed -// since the cache was populated, then the readFromStore callback is used to -// load new data. -func (a *AuthInfoCache) GetAuthInfo( +// GetAuthInfo consults the sessioninit.Cache and returns the AuthInfo for the +// provided username and databaseName. If the information is not in the cache, +// or if the underlying tables have changed since the cache was populated, +// then the readFromSystemTables callback is used to load new data. +func (a *Cache) GetAuthInfo( ctx context.Context, settings *cluster.Settings, ie sqlutil.InternalExecutor, @@ -131,18 +130,18 @@ func (a *AuthInfoCache) GetAuthInfo( if err != nil { return err } + + // If the underlying table versions are not committed, stop and avoid + // trying to cache anything. if usersTableDesc.IsUncommittedVersion() || - roleOptionsTableDesc.IsUncommittedVersion() || - !CacheEnabled.Get(&settings.SV) { + roleOptionsTableDesc.IsUncommittedVersion() { aInfo, err = readFromSystemTables( ctx, txn, ie, username, ) - if err != nil { - return err - } + return err } usersTableVersion := usersTableDesc.GetVersion() roleOptionsTableVersion := roleOptionsTableDesc.GetVersion() @@ -184,7 +183,7 @@ func (a *AuthInfoCache) GetAuthInfo( return aInfo, err } -func (a *AuthInfoCache) readAuthInfoFromCache( +func (a *Cache) readAuthInfoFromCache( ctx context.Context, usersTableVersion descpb.DescriptorVersion, roleOptionsTableVersion descpb.DescriptorVersion, @@ -194,7 +193,7 @@ func (a *AuthInfoCache) readAuthInfoFromCache( defer a.Unlock() // We don't need to check dbRoleSettingsTableVersion here, so pass in the // one we already have. - a.checkStaleness(ctx, usersTableVersion, roleOptionsTableVersion, a.dbRoleSettingsTableVersion) + a.clearCacheIfStale(ctx, usersTableVersion, roleOptionsTableVersion, a.dbRoleSettingsTableVersion) ai, foundAuthInfo := a.authInfoCache[username] return ai, foundAuthInfo } @@ -203,7 +202,7 @@ func (a *AuthInfoCache) readAuthInfoFromCache( // authInfoCache, and returns true if it succeeded. If the underlying system // tables have been modified since they were read, the authInfoCache is not // updated, and the passed in table descriptor versions are updated. -func (a *AuthInfoCache) writeAuthInfoBackToCache( +func (a *Cache) writeAuthInfoBackToCache( ctx context.Context, usersTableVersion *descpb.DescriptorVersion, roleOptionsTableVersion *descpb.DescriptorVersion, @@ -236,12 +235,12 @@ func (a *AuthInfoCache) writeAuthInfoBackToCache( return true } -// GetDefaultSettings consults the AuthInfoCache and returns the list of +// GetDefaultSettings consults the sessioninit.Cache and returns the list of // SettingsCacheEntry for the provided username and databaseName. If the // information is not in the cache, or if the underlying tables have changed -// since the cache was populated, then the readFromStore callback is used to -// load new data. -func (a *AuthInfoCache) GetDefaultSettings( +// since the cache was populated, then the readFromSystemTables callback is +// used to load new data. +func (a *Cache) GetDefaultSettings( ctx context.Context, settings *cluster.Settings, ie sqlutil.InternalExecutor, @@ -288,6 +287,11 @@ func (a *AuthInfoCache) GetDefaultSettings( } } + // If the underlying table versions are not committed or if the cache is + // disabled, stop and avoid trying to cache anything. + // We can't check if the cache is disabled earlier, since we always need to + // start the `CollectionFactory.Txn()` regardless in order to look up the + // database descriptor ID. if dbRoleSettingsTableDesc.IsUncommittedVersion() || !CacheEnabled.Get(&settings.SV) { settingsEntries, err = readFromSystemTables( ctx, @@ -296,9 +300,7 @@ func (a *AuthInfoCache) GetDefaultSettings( username, databaseID, ) - if err != nil || !CacheEnabled.Get(&settings.SV) { - return err - } + return err } dbRoleSettingsTableVersion := dbRoleSettingsTableDesc.GetVersion() @@ -338,7 +340,7 @@ func (a *AuthInfoCache) GetDefaultSettings( return settingsEntries, err } -func (a *AuthInfoCache) readDefaultSettingsFromCache( +func (a *Cache) readDefaultSettingsFromCache( ctx context.Context, dbRoleSettingsTableVersion descpb.DescriptorVersion, username security.SQLUsername, @@ -346,13 +348,17 @@ func (a *AuthInfoCache) readDefaultSettingsFromCache( ) ([]SettingsCacheEntry, bool) { a.Lock() defer a.Unlock() - // We don't need to check usersTableVersion or roleOptionsTableVersion here, \ + // We don't need to check usersTableVersion or roleOptionsTableVersion here, // so pass in the values we already have. - a.checkStaleness(ctx, a.usersTableVersion, a.roleOptionsTableVersion, dbRoleSettingsTableVersion) + a.clearCacheIfStale(ctx, a.usersTableVersion, a.roleOptionsTableVersion, dbRoleSettingsTableVersion) foundAllDefaultSettings := true var sEntries []SettingsCacheEntry - // Search through the cache for the settings entries we need. Note - // that GenerateSettingsCacheKeys goes in order of precedence. + // Search through the cache for the settings entries we need. Since we look up + // multiple entries in the cache, the same setting might appear multiple + // times. Note that GenerateSettingsCacheKeys goes in order of precedence, + // so the order of the returned []SettingsCacheEntry is important and the + // caller must take care not to apply a setting if it has already appeared + // earlier in the list. for _, k := range GenerateSettingsCacheKeys(databaseID, username) { s, ok := a.settingsCache[k] if !ok { @@ -364,12 +370,12 @@ func (a *AuthInfoCache) readDefaultSettingsFromCache( return sEntries, foundAllDefaultSettings } -// writeDefaultSettingsBackToCache tries to put the fetched SettingsCacheentry +// writeDefaultSettingsBackToCache tries to put the fetched SettingsCacheEntry // list into the settingsCache, and returns true if it succeeded. If the // underlying system tables have been modified since they were read, the // settingsCache is not updated, and the passed in table descriptor version is // updated. -func (a *AuthInfoCache) writeDefaultSettingsBackToCache( +func (a *Cache) writeDefaultSettingsBackToCache( ctx context.Context, dbRoleSettingsTableVersion *descpb.DescriptorVersion, settingsEntries []SettingsCacheEntry, @@ -414,7 +420,7 @@ func (a *AuthInfoCache) writeDefaultSettingsBackToCache( }() } -func (a *AuthInfoCache) checkStaleness( +func (a *Cache) clearCacheIfStale( ctx context.Context, usersTableVersion descpb.DescriptorVersion, roleOptionsTableVersion descpb.DescriptorVersion, diff --git a/pkg/sql/authentication/constants.go b/pkg/sql/sessioninit/constants.go similarity index 98% rename from pkg/sql/authentication/constants.go rename to pkg/sql/sessioninit/constants.go index 73de3043df1b..a1f917214ae1 100644 --- a/pkg/sql/authentication/constants.go +++ b/pkg/sql/sessioninit/constants.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package authentication +package sessioninit import ( "github.com/cockroachdb/cockroach/pkg/security" diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go index 32ceeca92794..64328a5a97d8 100644 --- a/pkg/sql/set_cluster_setting.go +++ b/pkg/sql/set_cluster_setting.go @@ -24,7 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -32,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -253,7 +253,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error { } } - if n.name == authentication.CacheEnabledSettingName { + if n.name == sessioninit.CacheEnabledSettingName { if expectedEncodedValue == "false" { // Bump role-related table versions to force other nodes to clear out // their AuthInfo cache. diff --git a/pkg/sql/user.go b/pkg/sql/user.go index e93cb5cb0bc1..ac2d4897cf48 100644 --- a/pkg/sql/user.go +++ b/pkg/sql/user.go @@ -17,10 +17,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/sql/authentication" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessioninit" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -28,14 +28,16 @@ import ( "github.com/cockroachdb/errors" ) -// GetUserAuthInfo determines if the given user exists and -// also returns a password retrieval function. +// GetUserSessionInitInfo determines if the given user exists and +// also returns a password retrieval function, other authentication-related +// information, and default session variable settings that are to be applied +// before a SQL session is created. // // The caller is responsible for normalizing the username. // (CockroachDB has case-insensitive usernames, unlike PostgreSQL.) // // The function is tolerant of unavailable clusters (or unavailable -// system.user) as follows: +// system database) as follows: // // - if the user is root, the user is reported to exist immediately // without querying system.users at all. The password retrieval @@ -49,20 +51,21 @@ import ( // ignored. This ensures that root has a modicum of comfort // logging into an unavailable cluster. // -// TODO(knz): this does not yet quite work becaus even if the pw +// TODO(knz): this does not yet quite work because even if the pw // auth on the UI succeeds writing to system.web_sessions will still // stall on an unavailable cluster and prevent root from logging in. // // - if the user is another user than root, then the function fails // after a timeout instead of blocking. The timeout is configurable -// via the cluster setting. +// via the cluster setting server.user_login.timeout. Note that this +// is a single timeout for looking up the password, role options, and +// default session variable settings. // -// - there is a cache for the the information from system.users and -// system.role_options. As long as the lookup succeeded before and there -// haven't been any CREATE/ALTER/DROP ROLE commands since, then the cache is -// used without a KV lookup. -// -func GetUserAuthInfo( +// - there is a cache for the the information from system.users, +// system.role_options, and system.database_role_settings. As long as the +// lookup succeeded before and there haven't been any CREATE/ALTER/DROP ROLE +// commands since, then the cache is used without a KV lookup. +func GetUserSessionInitInfo( ctx context.Context, execCfg *ExecutorConfig, ie *InternalExecutor, @@ -72,7 +75,7 @@ func GetUserAuthInfo( exists bool, canLogin bool, validUntil *tree.DTimestamp, - defaultSettings []authentication.SettingsCacheEntry, + defaultSettings []sessioninit.SettingsCacheEntry, pwRetrieveFn func(ctx context.Context) (hashedPassword []byte, err error), err error, ) { @@ -111,11 +114,7 @@ func retrieveSessionInitInfoWithCache( ie *InternalExecutor, username security.SQLUsername, databaseName string, -) ( - aInfo authentication.AuthInfo, - settingsEntries []authentication.SettingsCacheEntry, - err error, -) { +) (aInfo sessioninit.AuthInfo, settingsEntries []sessioninit.SettingsCacheEntry, err error) { // We may be operating with a timeout. timeout := userLoginTimeout.Get(&ie.s.cfg.Settings.SV) // We don't like long timeouts for root. @@ -132,7 +131,7 @@ func retrieveSessionInitInfoWithCache( } } err = runFn(func(ctx context.Context) (retErr error) { - aInfo, retErr = execCfg.AuthenticationInfoCache.GetAuthInfo( + aInfo, retErr = execCfg.SessionInitCache.GetAuthInfo( ctx, execCfg.Settings, ie, @@ -144,11 +143,11 @@ func retrieveSessionInitInfoWithCache( if retErr != nil { return retErr } - // Avoid lookingup default settings for root and non-existent users. + // Avoid looking up default settings for root and non-existent users. if username.IsRootUser() || !aInfo.UserExists { return nil } - settingsEntries, retErr = execCfg.AuthenticationInfoCache.GetDefaultSettings( + settingsEntries, retErr = execCfg.SessionInitCache.GetDefaultSettings( ctx, execCfg.Settings, ie, @@ -171,7 +170,7 @@ func retrieveSessionInitInfoWithCache( func retrieveAuthInfo( ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, username security.SQLUsername, -) (aInfo authentication.AuthInfo, retErr error) { +) (aInfo sessioninit.AuthInfo, retErr error) { // Use fully qualified table name to avoid looking up "".system.users. const getHashedPassword = `SELECT "hashedPassword" FROM system.public.users ` + `WHERE username=$1` @@ -180,7 +179,7 @@ func retrieveAuthInfo( sessiondata.InternalExecutorOverride{User: security.RootUserName()}, getHashedPassword, username) if err != nil { - return authentication.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", username) + return sessioninit.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", username) } if values != nil { aInfo.UserExists = true @@ -190,7 +189,7 @@ func retrieveAuthInfo( } if !aInfo.UserExists { - return authentication.AuthInfo{}, nil + return sessioninit.AuthInfo{}, nil } // None of the rest of the role options are relevant for root. @@ -209,7 +208,7 @@ func retrieveAuthInfo( username, ) if err != nil { - return authentication.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", username) + return sessioninit.AuthInfo{}, errors.Wrapf(err, "error looking up user %s", username) } // We have to make sure to close the iterator since we might return from // the for loop early (before Next() returns false). @@ -236,7 +235,7 @@ func retrieveAuthInfo( timeCtx := tree.NewParseTimeContext(timeutil.Now()) aInfo.ValidUntil, _, err = tree.ParseDTimestamp(timeCtx, ts, time.Microsecond) if err != nil { - return authentication.AuthInfo{}, errors.Wrap(err, + return sessioninit.AuthInfo{}, errors.Wrap(err, "error trying to parse timestamp while retrieving password valid until value") } } @@ -252,13 +251,13 @@ func retrieveDefaultSettings( ie sqlutil.InternalExecutor, username security.SQLUsername, databaseID descpb.ID, -) (settingsEntries []authentication.SettingsCacheEntry, retErr error) { +) (settingsEntries []sessioninit.SettingsCacheEntry, retErr error) { // Add an empty slice for all the keys so that something gets cached and // prevents a lookup for the same key from happening later. - keys := authentication.GenerateSettingsCacheKeys(databaseID, username) - settingsEntries = make([]authentication.SettingsCacheEntry, len(keys)) + keys := sessioninit.GenerateSettingsCacheKeys(databaseID, username) + settingsEntries = make([]sessioninit.SettingsCacheEntry, len(keys)) for i, k := range keys { - settingsEntries[i] = authentication.SettingsCacheEntry{ + settingsEntries[i] = sessioninit.SettingsCacheEntry{ SettingsCacheKey: k, Settings: []string{}, } @@ -306,7 +305,7 @@ WHERE fetchedSettings[i] = string(tree.MustBeDString(s)) } - thisKey := authentication.SettingsCacheKey{ + thisKey := sessioninit.SettingsCacheKey{ DatabaseID: fetechedDatabaseID, Username: fetchedUsername, } @@ -388,7 +387,7 @@ func (p *planner) BumpRoleMembershipTableVersion(ctx context.Context) error { // bumpUsersTableVersion increases the table version for the // users table. func (p *planner) bumpUsersTableVersion(ctx context.Context) error { - _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, authentication.UsersTableName, true, tree.ResolveAnyTableKind) + _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, sessioninit.UsersTableName, true, tree.ResolveAnyTableKind) if err != nil { return err } @@ -401,7 +400,7 @@ func (p *planner) bumpUsersTableVersion(ctx context.Context) error { // bumpRoleOptionsTableVersion increases the table version for the // role_options table. func (p *planner) bumpRoleOptionsTableVersion(ctx context.Context) error { - _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, authentication.RoleOptionsTableName, true, tree.ResolveAnyTableKind) + _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, sessioninit.RoleOptionsTableName, true, tree.ResolveAnyTableKind) if err != nil { return err } @@ -414,7 +413,7 @@ func (p *planner) bumpRoleOptionsTableVersion(ctx context.Context) error { // bumpDatabaseRoleSettingsTableVersion increases the table version for the // database_role_settings table. func (p *planner) bumpDatabaseRoleSettingsTableVersion(ctx context.Context) error { - _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, authentication.DatabaseRoleSettingsTableName, true, tree.ResolveAnyTableKind) + _, tableDesc, err := p.ResolveMutableTableDescriptor(ctx, sessioninit.DatabaseRoleSettingsTableName, true, tree.ResolveAnyTableKind) if err != nil { return err } From f983edb2276b029f6129e10855a6cc146a2f70cf Mon Sep 17 00:00:00 2001 From: Drew Kimball Date: Thu, 29 Jul 2021 18:39:55 -0600 Subject: [PATCH 4/8] sql: add cluster setting to preserve subquery and cte ordering This patch adds an experimental cluster setting `propagate_input_ordering` that indicates whether subqueries and CTEs should propagate their orderings to their parent scopes, when the parent scope is unordered. As an example, the following two queries should produce the following result when the cluster setting is true: ``` select * from (select * from generate_series(1, 10) i order by i % 5 asc, i asc) tmp; with tmp as (select * from generate_series(1, 10) i order by i % 5 asc, i asc) select * from tmp; ---- 5 10 1 6 2 7 3 8 4 9 ``` This allows cockroach to imitate postgres behavior - while postgres does not guarantee to maintain ordering on subqueries, it does in practice. Some existing applications take advantage of this fact, and so the ability to toggle this setting can help resolve incompatibilities in some cases. Fixes #68211 Release note: None --- pkg/sql/exec_util.go | 10 + .../testdata/logic_test/information_schema | 1 + .../logictest/testdata/logic_test/pg_catalog | 3 + .../testdata/logic_test/preserve_input_order | 137 ++++++++++ .../logictest/testdata/logic_test/show_source | 1 + pkg/sql/opt/optbuilder/project.go | 5 + pkg/sql/opt/optbuilder/select.go | 34 ++- .../optbuilder/testdata/preserve_input_order | 241 ++++++++++++++++++ pkg/sql/opt/optbuilder/with.go | 7 +- pkg/sql/opt/testutils/opttester/opt_tester.go | 8 + pkg/sql/sessiondata/session_data.go | 5 + pkg/sql/vars.go | 18 ++ 12 files changed, 466 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/preserve_input_order create mode 100644 pkg/sql/opt/optbuilder/testdata/preserve_input_order diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 968fa700b5f2..ec0e0d1c9873 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -432,6 +432,12 @@ var copyPartitioningWhenDeinterleavingTable = settings.RegisterBoolSetting( false, ).WithPublic() +var propagateInputOrdering = settings.RegisterBoolSetting( + `sql.defaults.propagate_input_ordering.enabled`, + `default value for the experimental propagate_input_ordering session variable`, + false, +) + // settingWorkMemBytes is a cluster setting that determines the maximum amount // of RAM that a processor can use. var settingWorkMemBytes = settings.RegisterByteSizeSetting( @@ -2637,6 +2643,10 @@ func (m *sessionDataMutator) SetExperimentalComputedColumnRewrites(val string) { m.data.ExperimentalComputedColumnRewrites = val } +func (m *sessionDataMutator) SetPropagateInputOrdering(b bool) { + m.data.PropagateInputOrdering = b +} + // Utility functions related to scrubbing sensitive information on SQL Stats. // quantizeCounts ensures that the Count field in the diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 55d69acae2f0..97f5deb0bb79 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -4561,6 +4561,7 @@ optimizer_use_histograms on optimizer_use_multicol_stats on override_multi_region_zone_config off prefer_lookup_joins_for_fks off +propagate_input_ordering off reorder_joins_limit 8 require_explicit_primary_keys off results_buffer_size 16384 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 385fc38a7104..6b35d192fcc6 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -3880,6 +3880,7 @@ optimizer_use_histograms on NULL optimizer_use_multicol_stats on NULL NULL NULL string override_multi_region_zone_config off NULL NULL NULL string prefer_lookup_joins_for_fks off NULL NULL NULL string +propagate_input_ordering off NULL NULL NULL string reorder_joins_limit 8 NULL NULL NULL string require_explicit_primary_keys off NULL NULL NULL string results_buffer_size 16384 NULL NULL NULL string @@ -3965,6 +3966,7 @@ optimizer_use_histograms on NULL optimizer_use_multicol_stats on NULL user NULL on on override_multi_region_zone_config off NULL user NULL off off prefer_lookup_joins_for_fks off NULL user NULL off off +propagate_input_ordering off NULL user NULL off off reorder_joins_limit 8 NULL user NULL 8 8 require_explicit_primary_keys off NULL user NULL off off results_buffer_size 16384 NULL user NULL 16384 16384 @@ -4047,6 +4049,7 @@ optimizer_use_histograms NULL NULL NULL optimizer_use_multicol_stats NULL NULL NULL NULL NULL override_multi_region_zone_config NULL NULL NULL NULL NULL prefer_lookup_joins_for_fks NULL NULL NULL NULL NULL +propagate_input_ordering NULL NULL NULL NULL NULL reorder_joins_limit NULL NULL NULL NULL NULL require_explicit_primary_keys NULL NULL NULL NULL NULL results_buffer_size NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/preserve_input_order b/pkg/sql/logictest/testdata/logic_test/preserve_input_order new file mode 100644 index 000000000000..9eb588bdc0d3 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/preserve_input_order @@ -0,0 +1,137 @@ +statement ok +SET propagate_input_ordering=true; + +query I +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) SELECT * FROM tmp; +---- +5 +10 +1 +6 +2 +7 +3 +8 +4 +9 + +query I +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) SELECT * FROM tmp; +---- +5 +10 +1 +6 +2 +7 +3 +8 +4 +9 + +# The internal ordering column for i%5 should not be present in the output. +query T +SELECT foo FROM (SELECT i, i%2 FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) AS foo +---- +(5,1) +(10,0) +(1,1) +(6,0) +(2,0) +(7,1) +(3,1) +(8,0) +(4,0) +(9,1) + +# The internal ordering column for i%5 should not be present in the output. +query II +SELECT foo.* FROM (SELECT i, i%2 FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) AS foo +---- +5 1 +10 0 +1 1 +6 0 +2 0 +7 1 +3 1 +8 0 +4 0 +9 1 + +# The subquery ordering is propagated to the aggregation. +query T +SELECT array_agg(i) FROM (SELECT * FROM generate_series(1, 5) i ORDER BY i%2 DESC, i) +---- +{1,3,5,2,4} + +# The input ordering is not propagated through joins. +query II +WITH tmp AS (SELECT * FROM generate_series(1, 2) x), + tmp2 AS (SELECT * FROM generate_series(8, 12) i ORDER BY i % 5 ASC, i ASC) +SELECT * FROM tmp, tmp2; +---- +1 8 +1 9 +1 10 +1 11 +1 12 +2 8 +2 9 +2 10 +2 11 +2 12 + +# The input ordering is not propagated through joins. +query II +SELECT * +FROM (SELECT * FROM generate_series(1, 2) x) tmp, + (SELECT * FROM generate_series(8, 12) i ORDER BY i % 5 ASC, i ASC) tmp2; +---- +1 8 +1 9 +1 10 +1 11 +1 12 +2 8 +2 9 +2 10 +2 11 +2 12 + +# Do not preserve the subquery ordering because the parent scope has its own +# ordering. +query I +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) +SELECT * FROM tmp ORDER BY i DESC; +---- +10 +9 +8 +7 +6 +5 +4 +3 +2 +1 + +# Do not preserve the subquery ordering because the parent scope has its own +# ordering. +query I +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) +SELECT * FROM tmp ORDER BY i DESC; +---- +10 +9 +8 +7 +6 +5 +4 +3 +2 +1 + +statement ok +RESET propagate_input_ordering; diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 8dae00618e97..48582c7b5913 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -77,6 +77,7 @@ optimizer_use_histograms on optimizer_use_multicol_stats on override_multi_region_zone_config off prefer_lookup_joins_for_fks off +propagate_input_ordering off reorder_joins_limit 8 require_explicit_primary_keys off results_buffer_size 16384 diff --git a/pkg/sql/opt/optbuilder/project.go b/pkg/sql/opt/optbuilder/project.go index 9f077ef2a6d7..33619e00a995 100644 --- a/pkg/sql/opt/optbuilder/project.go +++ b/pkg/sql/opt/optbuilder/project.go @@ -24,6 +24,11 @@ import ( // different set of columns than its input. Either way, it updates // projectionsScope.group with the output memo group ID. func (b *Builder) constructProjectForScope(inScope, projectionsScope *scope) { + if b.evalCtx.SessionData.PropagateInputOrdering && len(projectionsScope.ordering) == 0 { + // Preserve the input ordering. + projectionsScope.copyOrdering(inScope) + } + // Don't add an unnecessary "pass through" project. if projectionsScope.hasSameColumns(inScope) { projectionsScope.expr = inScope.expr diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 2ba6ec3e0af8..7e0ac315b7f0 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -77,9 +77,9 @@ func (b *Builder) buildDataSource( if cte := inScope.resolveCTE(tn); cte != nil { locking.ignoreLockingForCTE() outScope = inScope.push() - inCols := make(opt.ColList, len(cte.cols)) - outCols := make(opt.ColList, len(cte.cols)) - outScope.cols = nil + inCols := make(opt.ColList, len(cte.cols), len(cte.cols)+len(inScope.ordering)) + outCols := make(opt.ColList, len(cte.cols), len(cte.cols)+len(inScope.ordering)) + outScope.cols, outScope.extraCols = nil, nil for i, col := range cte.cols { id := col.ID c := b.factory.Metadata().ColumnMeta(id) @@ -89,6 +89,34 @@ func (b *Builder) buildDataSource( outCols[i] = newCol.id } + if b.evalCtx.SessionData.PropagateInputOrdering && len(inScope.ordering) > 0 { + var oldToNew opt.ColMap + for i := range inCols { + oldToNew.Set(int(inCols[i]), int(outCols[i])) + } + outScope.ordering = make(opt.Ordering, len(inScope.ordering)) + for i, col := range inScope.ordering { + var newID int + var ok bool + if newID, ok = oldToNew.Get(int(col.ID())); !ok { + c := b.factory.Metadata().ColumnMeta(col.ID()) + outScope.extraCols = append(outScope.extraCols, + scopeColumn{ + name: scopeColName(tree.Name("order_" + c.Alias)), + typ: c.Type, + }, + ) + newCol := &outScope.extraCols[len(outScope.extraCols)-1] + b.populateSynthesizedColumn(newCol, nil) + newCol.table = *tn + newID = int(newCol.id) + inCols = append(inCols, col.ID()) + outCols = append(outCols, newCol.id) + } + outScope.ordering[i] = opt.MakeOrderingColumn(opt.ColumnID(newID), col.Descending()) + } + } + outScope.expr = b.factory.ConstructWithScan(&memo.WithScanPrivate{ With: cte.id, Name: string(cte.name.Alias), diff --git a/pkg/sql/opt/optbuilder/testdata/preserve_input_order b/pkg/sql/opt/optbuilder/testdata/preserve_input_order new file mode 100644 index 000000000000..1760db9bfc38 --- /dev/null +++ b/pkg/sql/opt/optbuilder/testdata/preserve_input_order @@ -0,0 +1,241 @@ +# tests adapted from logictest -- preserve_input_order + +build preserve-input-order +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) SELECT * FROM tmp; +---- +sort + ├── columns: i:3 [hidden: order_column2:4] + ├── ordering: +4,+3 + └── with &1 (tmp) + ├── columns: i:3 order_column2:4 + ├── project + │ ├── columns: column2:2 generate_series:1 + │ ├── project-set + │ │ ├── columns: generate_series:1 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(1, 10) + │ └── projections + │ └── generate_series:1 % 5 [as=column2:2] + └── with-scan &1 (tmp) + ├── columns: i:3 order_column2:4 + └── mapping: + ├── generate_series:1 => i:3 + └── column2:2 => order_column2:4 + +build preserve-input-order +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) SELECT * FROM tmp; +---- +sort + ├── columns: i:3 [hidden: order_column2:4] + ├── ordering: +4,+3 + └── with &1 (tmp) + ├── columns: i:3 order_column2:4 + ├── project + │ ├── columns: column2:2 generate_series:1 + │ ├── project-set + │ │ ├── columns: generate_series:1 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(1, 10) + │ └── projections + │ └── generate_series:1 % 5 [as=column2:2] + └── with-scan &1 (tmp) + ├── columns: i:3 order_column2:4 + └── mapping: + ├── generate_series:1 => i:3 + └── column2:2 => order_column2:4 + +# The internal ordering column for i%5 should not be present in the output. +build preserve-input-order +SELECT foo FROM (SELECT i, i%2 FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) AS foo +---- +sort + ├── columns: foo:4 [hidden: generate_series:1 column3:3] + ├── ordering: +3,+1 + └── project + ├── columns: foo:4 generate_series:1 column3:3 + ├── project + │ ├── columns: "?column?":2 column3:3 generate_series:1 + │ ├── project-set + │ │ ├── columns: generate_series:1 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(1, 10) + │ └── projections + │ ├── generate_series:1 % 2 [as="?column?":2] + │ └── generate_series:1 % 5 [as=column3:3] + └── projections + └── ((generate_series:1, "?column?":2) AS i, "?column?") [as=foo:4] + +# The internal ordering column for i%5 should not be present in the output. +build preserve-input-order +SELECT foo.* FROM (SELECT i, i%2 FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) AS foo +---- +sort + ├── columns: i:1 "?column?":2 [hidden: column3:3] + ├── ordering: +3,+1 + └── project + ├── columns: "?column?":2 column3:3 generate_series:1 + ├── project-set + │ ├── columns: generate_series:1 + │ ├── values + │ │ └── () + │ └── zip + │ └── generate_series(1, 10) + └── projections + ├── generate_series:1 % 2 [as="?column?":2] + └── generate_series:1 % 5 [as=column3:3] + +# The subquery ordering is propagated to the aggregation. +build preserve-input-order +SELECT array_agg(i) FROM (SELECT * FROM generate_series(1, 5) i ORDER BY i%2 DESC, i) +---- +scalar-group-by + ├── columns: array_agg:3 + ├── internal-ordering: -2,+1 + ├── sort + │ ├── columns: generate_series:1 column2:2 + │ ├── ordering: -2,+1 + │ └── project + │ ├── columns: column2:2 generate_series:1 + │ ├── project-set + │ │ ├── columns: generate_series:1 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(1, 5) + │ └── projections + │ └── generate_series:1 % 2 [as=column2:2] + └── aggregations + └── array-agg [as=array_agg:3] + └── generate_series:1 + +# The input ordering is not propagated through joins. +build preserve-input-order +WITH tmp AS (SELECT * FROM generate_series(1, 2) x), + tmp2 AS (SELECT * FROM generate_series(8, 12) i ORDER BY i % 5 ASC, i ASC) +SELECT * FROM tmp INNER JOIN tmp2 ON True; +---- +with &1 (tmp) + ├── columns: x:4 i:7 [hidden: order_column3:5 order_generate_series:6 order_column3:8] + ├── project-set + │ ├── columns: generate_series:1 + │ ├── values + │ │ └── () + │ └── zip + │ └── generate_series(1, 2) + └── with &2 (tmp2) + ├── columns: x:4 order_column3:5 order_generate_series:6 i:7 order_column3:8 + ├── project + │ ├── columns: column3:3 generate_series:2 + │ ├── project-set + │ │ ├── columns: generate_series:2 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(8, 12) + │ └── projections + │ └── generate_series:2 % 5 [as=column3:3] + └── inner-join (cross) + ├── columns: x:4 order_column3:5 order_generate_series:6 i:7 order_column3:8 + ├── with-scan &1 (tmp) + │ ├── columns: x:4 order_column3:5 order_generate_series:6 + │ └── mapping: + │ ├── generate_series:1 => x:4 + │ ├── column3:3 => order_column3:5 + │ └── generate_series:2 => order_generate_series:6 + ├── with-scan &2 (tmp2) + │ ├── columns: i:7 order_column3:8 + │ └── mapping: + │ ├── generate_series:2 => i:7 + │ └── column3:3 => order_column3:8 + └── filters + └── true + +# The input ordering is not propagated through joins. +build preserve-input-order +SELECT * +FROM (SELECT * FROM generate_series(1, 2) x) tmp, + (SELECT * FROM generate_series(8, 12) i ORDER BY i % 5 ASC, i ASC) tmp2; +---- +inner-join (cross) + ├── columns: x:1 i:2 [hidden: column3:3] + ├── project-set + │ ├── columns: generate_series:1 + │ ├── values + │ │ └── () + │ └── zip + │ └── generate_series(1, 2) + ├── project + │ ├── columns: column3:3 generate_series:2 + │ ├── project-set + │ │ ├── columns: generate_series:2 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(8, 12) + │ └── projections + │ └── generate_series:2 % 5 [as=column3:3] + └── filters (true) + +# Do not preserve the subquery ordering because the parent scope has its own +# ordering. +build preserve-input-order +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) +SELECT * FROM tmp ORDER BY i DESC; +---- +sort + ├── columns: i:3 + ├── ordering: -3 + └── with &1 (tmp) + ├── columns: i:3 + ├── project + │ ├── columns: column2:2 generate_series:1 + │ ├── project-set + │ │ ├── columns: generate_series:1 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(1, 10) + │ └── projections + │ └── generate_series:1 % 5 [as=column2:2] + └── project + ├── columns: i:3 + └── with-scan &1 (tmp) + ├── columns: i:3 order_column2:4 + └── mapping: + ├── generate_series:1 => i:3 + └── column2:2 => order_column2:4 + +# Do not preserve the subquery ordering because the parent scope has its own +# ordering. +build preserve-input-order +WITH tmp AS (SELECT * FROM generate_series(1, 10) i ORDER BY i % 5 ASC, i ASC) +SELECT * FROM tmp ORDER BY i DESC; +---- +sort + ├── columns: i:3 + ├── ordering: -3 + └── with &1 (tmp) + ├── columns: i:3 + ├── project + │ ├── columns: column2:2 generate_series:1 + │ ├── project-set + │ │ ├── columns: generate_series:1 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── generate_series(1, 10) + │ └── projections + │ └── generate_series:1 % 5 [as=column2:2] + └── project + ├── columns: i:3 + └── with-scan &1 (tmp) + ├── columns: i:3 order_column2:4 + └── mapping: + ├── generate_series:1 => i:3 + └── column2:2 => order_column2:4 diff --git a/pkg/sql/opt/optbuilder/with.go b/pkg/sql/opt/optbuilder/with.go index acf64bc71ed6..88040975b9dc 100644 --- a/pkg/sql/opt/optbuilder/with.go +++ b/pkg/sql/opt/optbuilder/with.go @@ -186,7 +186,12 @@ func (b *Builder) buildCTE( if !isRecursive { cteScope := b.buildStmt(cte.Stmt, nil /* desiredTypes */, inScope) cteScope.removeHiddenCols() - b.dropOrderingAndExtraCols(cteScope) + if b.evalCtx.SessionData.PropagateInputOrdering && len(inScope.ordering) == 0 { + // Preserve the CTE ordering. + inScope.copyOrdering(cteScope) + } else { + b.dropOrderingAndExtraCols(cteScope) + } return cteScope.expr, b.getCTECols(cteScope, cte.Name) } diff --git a/pkg/sql/opt/testutils/opttester/opt_tester.go b/pkg/sql/opt/testutils/opttester/opt_tester.go index 904e3016e057..13a555d815e4 100644 --- a/pkg/sql/opt/testutils/opttester/opt_tester.go +++ b/pkg/sql/opt/testutils/opttester/opt_tester.go @@ -173,6 +173,10 @@ type Flags struct { // SessionData.PreferLookupJoinsForFKs. PreferLookupJoinsForFKs bool + // PropagateInputOrdering is the default value for + // SessionData.PropagateInputOrdering + PropagateInputOrdering bool + // Locality specifies the location of the planning node as a set of user- // defined key/value pairs, ordered from most inclusive to least inclusive. // If there are no tiers, then the node's location is not known. Examples: @@ -477,6 +481,7 @@ func (ot *OptTester) RunCommand(tb testing.TB, d *datadriven.TestData) string { ot.evalCtx.SessionData.ReorderJoinsLimit = ot.Flags.JoinLimit ot.evalCtx.SessionData.PreferLookupJoinsForFKs = ot.Flags.PreferLookupJoinsForFKs + ot.evalCtx.SessionData.PropagateInputOrdering = ot.Flags.PropagateInputOrdering ot.evalCtx.TestingKnobs.OptimizerCostPerturbation = ot.Flags.PerturbCost ot.evalCtx.Locality = ot.Flags.Locality @@ -988,6 +993,9 @@ func (f *Flags) Set(arg datadriven.CmdArg) error { case "query-args": f.QueryArgs = arg.Vals + case "preserve-input-order": + f.PropagateInputOrdering = true + default: return fmt.Errorf("unknown argument: %s", arg.Key) } diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go index 860de33f24c3..8b3662c6b863 100644 --- a/pkg/sql/sessiondata/session_data.go +++ b/pkg/sql/sessiondata/session_data.go @@ -285,6 +285,11 @@ type LocalOnlySessionData struct { // of that interleave should be applied to the new primary index. CopyPartitioningWhenDeinterleavingTable bool + // PropagateInputOrdering indicates that when planning a subquery or CTE, the + // inner ordering should be propagated to the outer scope if the outer scope + // is unordered. PropagateInputOrdering is currently experimental. + PropagateInputOrdering bool + /////////////////////////////////////////////////////////////////////////// // WARNING: consider whether a session parameter you're adding needs to // // be propagated to the remote nodes. If so, that parameter should live // diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index e4f2721f7796..911d07111308 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -1437,6 +1437,24 @@ var varGen = map[string]sessionVar{ return formatBoolAsPostgresSetting(copyPartitioningWhenDeinterleavingTable.Get(sv)) }, }, + + `propagate_input_ordering`: { + GetStringVal: makePostgresBoolGetStringValFn(`propagate_input_ordering`), + Set: func(_ context.Context, m *sessionDataMutator, s string) error { + b, err := paramparse.ParseBoolVar(`propagate_input_ordering`, s) + if err != nil { + return err + } + m.SetPropagateInputOrdering(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext) string { + return formatBoolAsPostgresSetting(evalCtx.SessionData.PropagateInputOrdering) + }, + GlobalDefault: func(sv *settings.Values) string { + return formatBoolAsPostgresSetting(propagateInputOrdering.Get(sv)) + }, + }, } const compatErrMsg = "this parameter is currently recognized only for compatibility and has no effect in CockroachDB." From c69c62b00ba9dc50f7cd00a73b170df259f7de42 Mon Sep 17 00:00:00 2001 From: richardjcai Date: Wed, 4 Aug 2021 18:55:49 -0400 Subject: [PATCH 5/8] sql: refactor default privileges for all roles Default privileges for all roles are refactored to be more in line with regular roles. Simplifies some code. Release note: None --- pkg/sql/alter_default_privileges.go | 8 +- pkg/sql/catalog/descpb/default_privilege.go | 121 +++-- pkg/sql/catalog/descpb/privilege.pb.go | 431 +++++++++--------- pkg/sql/catalog/descpb/privilege.proto | 10 +- pkg/sql/crdb_internal.go | 6 +- pkg/sql/logictest/testdata/logic_test/bytes | 2 +- .../logic_test/pg_catalog_pg_default_acl | 30 ++ .../testdata/show_trace_nonmetamorphic | 2 +- pkg/sql/pg_catalog.go | 14 +- 9 files changed, 356 insertions(+), 268 deletions(-) diff --git a/pkg/sql/alter_default_privileges.go b/pkg/sql/alter_default_privileges.go index 06ff8406269d..b96c9052f28e 100644 --- a/pkg/sql/alter_default_privileges.go +++ b/pkg/sql/alter_default_privileges.go @@ -135,15 +135,15 @@ func (n *alterDefaultPrivilegesNode) startExec(params runParams) error { defaultPrivs := n.dbDesc.GetDefaultPrivileges() - var roles []descpb.AlterDefaultPrivilegesRole + var roles []descpb.DefaultPrivilegesRole if n.n.ForAllRoles { - roles = append(roles, descpb.AlterDefaultPrivilegesRole{ + roles = append(roles, descpb.DefaultPrivilegesRole{ ForAllRoles: true, }) } else { - roles = make([]descpb.AlterDefaultPrivilegesRole, len(targetRoles)) + roles = make([]descpb.DefaultPrivilegesRole, len(targetRoles)) for i, role := range targetRoles { - roles[i] = descpb.AlterDefaultPrivilegesRole{ + roles[i] = descpb.DefaultPrivilegesRole{ Role: role, } } diff --git a/pkg/sql/catalog/descpb/default_privilege.go b/pkg/sql/catalog/descpb/default_privilege.go index b43a9dbd5af4..68ffb9ebd534 100644 --- a/pkg/sql/catalog/descpb/default_privilege.go +++ b/pkg/sql/catalog/descpb/default_privilege.go @@ -20,29 +20,24 @@ import ( "github.com/cockroachdb/errors" ) -// AlterDefaultPrivilegesRole represents the creator role that the default privileges +// DefaultPrivilegesRole represents the creator role that the default privileges // are being altered for. // Either: // role should be populated // forAllRoles should be true. -type AlterDefaultPrivilegesRole struct { +type DefaultPrivilegesRole struct { Role security.SQLUsername ForAllRoles bool } // GrantDefaultPrivileges grants privileges for the specified users. func (p *DefaultPrivilegeDescriptor) GrantDefaultPrivileges( - role AlterDefaultPrivilegesRole, + role DefaultPrivilegesRole, privileges privilege.List, grantees tree.NameList, targetObject tree.AlterDefaultPrivilegesTargetObject, ) { - var defaultPrivilegesPerObject map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor - if role.ForAllRoles { - defaultPrivilegesPerObject = p.DefaultPrivilegesForAllRoles - } else { - defaultPrivilegesPerObject = p.findOrCreateUser(role.Role).DefaultPrivilegesPerObject - } + defaultPrivilegesPerObject := p.findOrCreateUser(role).DefaultPrivilegesPerObject for _, grantee := range grantees { defaultPrivileges := defaultPrivilegesPerObject[targetObject] defaultPrivileges.Grant( @@ -55,17 +50,12 @@ func (p *DefaultPrivilegeDescriptor) GrantDefaultPrivileges( // RevokeDefaultPrivileges revokes privileges for the specified users. func (p *DefaultPrivilegeDescriptor) RevokeDefaultPrivileges( - role AlterDefaultPrivilegesRole, + role DefaultPrivilegesRole, privileges privilege.List, grantees tree.NameList, targetObject tree.AlterDefaultPrivilegesTargetObject, ) { - var defaultPrivilegesPerObject map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor - if role.ForAllRoles { - defaultPrivilegesPerObject = p.DefaultPrivilegesForAllRoles - } else { - defaultPrivilegesPerObject = p.findOrCreateUser(role.Role).DefaultPrivilegesPerObject - } + defaultPrivilegesPerObject := p.findOrCreateUser(role).DefaultPrivilegesPerObject for _, grantee := range grantees { defaultPrivileges := defaultPrivilegesPerObject[targetObject] defaultPrivileges.Revoke( @@ -98,23 +88,30 @@ func CreatePrivilegesFromDefaultPrivileges( defaultPrivileges = InitDefaultPrivilegeDescriptor() } - defaultPrivilegesForAllRoles := defaultPrivileges.DefaultPrivilegesForAllRoles - // The privileges for the object are the union of the default privileges // defined for the object for the object creator and the default privileges // defined for all roles. newPrivs := NewDefaultPrivilegeDescriptor(user) - privilegesForAllRoles, found := defaultPrivilegesForAllRoles[targetObject] + defaultPrivilegesForAllRoles, found := defaultPrivileges.GetDefaultPrivilegesForRole( + DefaultPrivilegesRole{ + ForAllRoles: true, + }, + ) if found { - for _, user := range privilegesForAllRoles.Users { - newPrivs.Grant( - user.UserProto.Decode(), - privilege.ListFromBitField(user.Privileges, targetObject.ToPrivilegeObjectType()), - ) + defaultPrivileges, descriptorExists := defaultPrivilegesForAllRoles.DefaultPrivilegesPerObject[targetObject] + if descriptorExists { + for _, user := range defaultPrivileges.Users { + newPrivs.Grant( + user.UserProto.Decode(), + privilege.ListFromBitField(user.Privileges, targetObject.ToPrivilegeObjectType()), + ) + } } } - defaultPrivilegesForCreator, defaultPrivilegesDefinedForCreator := defaultPrivileges.GetDefaultPrivilegesForRole(user) + defaultPrivilegesForCreator, defaultPrivilegesDefinedForCreator := defaultPrivileges.GetDefaultPrivilegesForRole(DefaultPrivilegesRole{ + Role: user, + }) if defaultPrivilegesDefinedForCreator { defaultPrivileges, descriptorExists := defaultPrivilegesForCreator.DefaultPrivilegesPerObject[targetObject] if descriptorExists { @@ -145,17 +142,42 @@ func CreatePrivilegesFromDefaultPrivileges( return newPrivs } -// User accesses the role field. -func (u *DefaultPrivilegesForRole) User() security.SQLUsername { - return u.UserProto.Decode() +// ToDefaultPrivilegesRole returns the DefaultPrivilegesRole corresponding to +// DefaultPrivilegesForRole. +func (u *DefaultPrivilegesForRole) ToDefaultPrivilegesRole() DefaultPrivilegesRole { + if u.GetForAllRoles() { + return DefaultPrivilegesRole{ + ForAllRoles: true, + } + } + return DefaultPrivilegesRole{ + Role: u.GetUserProto().Decode(), + } +} + +// LessThan returns whether r is less than other. +// The DefaultPrivilegesRole with ForAllRoles set is always considered +// larger. Only one of r or other should have ForAllRoles set since there +// should only ever be one entry for all roles. +// If ForAllRoles is set for neither, we do a string comparison on the username. +func (r DefaultPrivilegesRole) LessThan(other DefaultPrivilegesRole) bool { + // Defined such that ForAllRoles is never less than. + if r.ForAllRoles { + return false + } + if other.ForAllRoles { + return true + } + + return r.Role.LessThan(other.Role) } // GetDefaultPrivilegesForRole looks for a specific user in the list. // Returns (nil, false) if not found, or (obj, true) if found. func (p *DefaultPrivilegeDescriptor) GetDefaultPrivilegesForRole( - user security.SQLUsername, + role DefaultPrivilegesRole, ) (*DefaultPrivilegesForRole, bool) { - idx := p.findUserIndex(user) + idx := p.findUserIndex(role) if idx == -1 { return nil, false } @@ -164,11 +186,12 @@ func (p *DefaultPrivilegeDescriptor) GetDefaultPrivilegesForRole( // findUserIndex looks for a given user and returns its // index in the User array if found. Returns -1 otherwise. -func (p *DefaultPrivilegeDescriptor) findUserIndex(user security.SQLUsername) int { +func (p *DefaultPrivilegeDescriptor) findUserIndex(role DefaultPrivilegesRole) int { idx := sort.Search(len(p.DefaultPrivilegesPerRole), func(i int) bool { - return !p.DefaultPrivilegesPerRole[i].User().LessThan(user) + return !p.DefaultPrivilegesPerRole[i].ToDefaultPrivilegesRole().LessThan(role) }) - if idx < len(p.DefaultPrivilegesPerRole) && p.DefaultPrivilegesPerRole[idx].User() == user { + if idx < len(p.DefaultPrivilegesPerRole) && + p.DefaultPrivilegesPerRole[idx].ToDefaultPrivilegesRole() == role { return idx } return -1 @@ -178,27 +201,33 @@ func (p *DefaultPrivilegeDescriptor) findUserIndex(user security.SQLUsername) in // If a new user is created, it must be added in the correct sorted order // in the list. func (p *DefaultPrivilegeDescriptor) findOrCreateUser( - user security.SQLUsername, + role DefaultPrivilegesRole, ) *DefaultPrivilegesForRole { idx := sort.Search(len(p.DefaultPrivilegesPerRole), func(i int) bool { - return !p.DefaultPrivilegesPerRole[i].User().LessThan(user) + return !p.DefaultPrivilegesPerRole[i].ToDefaultPrivilegesRole().LessThan(role) }) + var defaultPrivilegeRole isDefaultPrivilegesForRole_Role + if role.ForAllRoles { + defaultPrivilegeRole = &DefaultPrivilegesForRole_ForAllRoles{ForAllRoles: true} + } else { + defaultPrivilegeRole = &DefaultPrivilegesForRole_UserProto{UserProto: role.Role.EncodeProto()} + } if idx == len(p.DefaultPrivilegesPerRole) { // Not found but should be inserted at the end. p.DefaultPrivilegesPerRole = append(p.DefaultPrivilegesPerRole, DefaultPrivilegesForRole{ - UserProto: user.EncodeProto(), + Role: defaultPrivilegeRole, DefaultPrivilegesPerObject: map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor{}, }, ) - } else if p.DefaultPrivilegesPerRole[idx].User() == user { + } else if p.DefaultPrivilegesPerRole[idx].ToDefaultPrivilegesRole() == role { // Found. } else { // New element to be inserted at idx. p.DefaultPrivilegesPerRole = append(p.DefaultPrivilegesPerRole, DefaultPrivilegesForRole{}) copy(p.DefaultPrivilegesPerRole[idx+1:], p.DefaultPrivilegesPerRole[idx:]) p.DefaultPrivilegesPerRole[idx] = DefaultPrivilegesForRole{ - UserProto: user.EncodeProto(), + Role: defaultPrivilegeRole, DefaultPrivilegesPerObject: map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor{}, } } @@ -208,9 +237,17 @@ func (p *DefaultPrivilegeDescriptor) findOrCreateUser( // Validate returns an assertion error if the default privilege descriptor // is invalid. func (p *DefaultPrivilegeDescriptor) Validate() error { + entryForAllRolesFound := false for i, defaultPrivilegesForRole := range p.DefaultPrivilegesPerRole { + if defaultPrivilegesForRole.GetForAllRoles() { + if entryForAllRolesFound { + return errors.AssertionFailedf("multiple entries found in map for all roles") + } + entryForAllRolesFound = true + } if i+1 < len(p.DefaultPrivilegesPerRole) && - !defaultPrivilegesForRole.User().LessThan(p.DefaultPrivilegesPerRole[i+1].User()) { + !defaultPrivilegesForRole.ToDefaultPrivilegesRole(). + LessThan(p.DefaultPrivilegesPerRole[i+1].ToDefaultPrivilegesRole()) { return errors.AssertionFailedf("default privilege list is not sorted") } for objectType, defaultPrivileges := range defaultPrivilegesForRole.DefaultPrivilegesPerObject { @@ -231,11 +268,5 @@ func InitDefaultPrivilegeDescriptor() *DefaultPrivilegeDescriptor { var defaultPrivilegesForRole []DefaultPrivilegesForRole return &DefaultPrivilegeDescriptor{ DefaultPrivilegesPerRole: defaultPrivilegesForRole, - DefaultPrivilegesForAllRoles: map[tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor{ - tree.Tables: {}, - tree.Sequences: {}, - tree.Types: {}, - tree.Schemas: {}, - }, } } diff --git a/pkg/sql/catalog/descpb/privilege.pb.go b/pkg/sql/catalog/descpb/privilege.pb.go index 4caa29e9bed4..3c27880e3c8b 100644 --- a/pkg/sql/catalog/descpb/privilege.pb.go +++ b/pkg/sql/catalog/descpb/privilege.pb.go @@ -112,8 +112,11 @@ var xxx_messageInfo_PrivilegeDescriptor proto.InternalMessageInfo // and it's value is a PrivilegeDescriptor that is only used for // the list of UserPrivileges for that object. type DefaultPrivilegesForRole struct { - UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,1,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"` - DefaultPrivilegesPerObject map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor `protobuf:"bytes,2,rep,name=default_privileges_per_object,json=defaultPrivilegesPerObject,castkey=github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject" json:"default_privileges_per_object" protobuf_key:"varint,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + // Types that are valid to be assigned to Role: + // *DefaultPrivilegesForRole_UserProto + // *DefaultPrivilegesForRole_ForAllRoles + Role isDefaultPrivilegesForRole_Role `protobuf_oneof:"role"` + DefaultPrivilegesPerObject map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor `protobuf:"bytes,3,rep,name=default_privileges_per_object,json=defaultPrivilegesPerObject,castkey=github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject" json:"default_privileges_per_object" protobuf_key:"varint,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` } func (m *DefaultPrivilegesForRole) Reset() { *m = DefaultPrivilegesForRole{} } @@ -145,12 +148,57 @@ func (m *DefaultPrivilegesForRole) XXX_DiscardUnknown() { var xxx_messageInfo_DefaultPrivilegesForRole proto.InternalMessageInfo +type isDefaultPrivilegesForRole_Role interface { + isDefaultPrivilegesForRole_Role() + Equal(interface{}) bool + MarshalTo([]byte) (int, error) + Size() int +} + +type DefaultPrivilegesForRole_UserProto struct { + UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,1,opt,name=user_proto,json=userProto,oneof,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto,omitempty"` +} +type DefaultPrivilegesForRole_ForAllRoles struct { + ForAllRoles bool `protobuf:"varint,2,opt,name=for_all_roles,json=forAllRoles,oneof" json:"for_all_roles,omitempty"` +} + +func (*DefaultPrivilegesForRole_UserProto) isDefaultPrivilegesForRole_Role() {} +func (*DefaultPrivilegesForRole_ForAllRoles) isDefaultPrivilegesForRole_Role() {} + +func (m *DefaultPrivilegesForRole) GetRole() isDefaultPrivilegesForRole_Role { + if m != nil { + return m.Role + } + return nil +} + +func (m *DefaultPrivilegesForRole) GetUserProto() github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto { + if x, ok := m.GetRole().(*DefaultPrivilegesForRole_UserProto); ok { + return x.UserProto + } + return "" +} + +func (m *DefaultPrivilegesForRole) GetForAllRoles() bool { + if x, ok := m.GetRole().(*DefaultPrivilegesForRole_ForAllRoles); ok { + return x.ForAllRoles + } + return false +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*DefaultPrivilegesForRole) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*DefaultPrivilegesForRole_UserProto)(nil), + (*DefaultPrivilegesForRole_ForAllRoles)(nil), + } +} + // DefaultPrivilegeDescriptor describes the set of default privileges for a // given role + object type. // The DefaultPrivileges list must be sorted for fast access and user lookups. type DefaultPrivilegeDescriptor struct { - DefaultPrivilegesPerRole []DefaultPrivilegesForRole `protobuf:"bytes,1,rep,name=default_privileges_per_role,json=defaultPrivilegesPerRole" json:"default_privileges_per_role"` - DefaultPrivilegesForAllRoles map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor `protobuf:"bytes,2,rep,name=default_privileges_for_all_roles,json=defaultPrivilegesForAllRoles,castkey=github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject" json:"default_privileges_for_all_roles" protobuf_key:"varint,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + DefaultPrivilegesPerRole []DefaultPrivilegesForRole `protobuf:"bytes,1,rep,name=default_privileges_per_role,json=defaultPrivilegesPerRole" json:"default_privileges_per_role"` } func (m *DefaultPrivilegeDescriptor) Reset() { *m = DefaultPrivilegeDescriptor{} } @@ -188,7 +236,6 @@ func init() { proto.RegisterType((*DefaultPrivilegesForRole)(nil), "cockroach.sql.sqlbase.DefaultPrivilegesForRole") proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor)(nil), "cockroach.sql.sqlbase.DefaultPrivilegesForRole.DefaultPrivilegesPerObjectEntry") proto.RegisterType((*DefaultPrivilegeDescriptor)(nil), "cockroach.sql.sqlbase.DefaultPrivilegeDescriptor") - proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor)(nil), "cockroach.sql.sqlbase.DefaultPrivilegeDescriptor.DefaultPrivilegesForAllRolesEntry") } func init() { @@ -196,43 +243,42 @@ func init() { } var fileDescriptor_9343d951995d5a76 = []byte{ - // 568 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x55, 0x4f, 0x6b, 0x13, 0x41, - 0x14, 0xcf, 0x24, 0xad, 0xd2, 0x17, 0x2a, 0xb2, 0x56, 0x58, 0x62, 0xdd, 0xc4, 0xa0, 0x10, 0x3c, - 0xec, 0x42, 0x0f, 0x22, 0x3d, 0x88, 0x09, 0xb5, 0x27, 0xa1, 0x71, 0x5b, 0x3d, 0x88, 0x10, 0x36, - 0x9b, 0xd7, 0xed, 0x9a, 0x49, 0x66, 0x33, 0x33, 0x1b, 0x09, 0xde, 0xfc, 0x04, 0xfa, 0x0d, 0x04, - 0xbf, 0x4c, 0x8e, 0x3d, 0xf6, 0x54, 0x35, 0xb9, 0x78, 0xf4, 0xe2, 0x41, 0x2f, 0xca, 0xcc, 0x6e, - 0x93, 0xd6, 0x26, 0xb6, 0x82, 0xf6, 0xb0, 0x30, 0xec, 0xbe, 0xf9, 0xfd, 0x79, 0xbf, 0xf7, 0x58, - 0x28, 0x8b, 0x1e, 0x75, 0x7c, 0x4f, 0x7a, 0x94, 0x05, 0x4e, 0x0b, 0x85, 0x1f, 0x35, 0x9d, 0x88, - 0x87, 0xfd, 0x90, 0x62, 0x80, 0x76, 0xc4, 0x99, 0x64, 0xc6, 0x75, 0x9f, 0xf9, 0x6d, 0xce, 0x3c, - 0x7f, 0xcf, 0x16, 0x3d, 0xaa, 0x9e, 0xa6, 0x27, 0xb0, 0xb0, 0x12, 0xb0, 0x80, 0xe9, 0x0a, 0x47, - 0x9d, 0x92, 0xe2, 0xf2, 0x07, 0x02, 0x57, 0x9e, 0x0a, 0xe4, 0xf5, 0x23, 0x10, 0x61, 0x20, 0x40, - 0x2c, 0x90, 0x37, 0x74, 0x81, 0x49, 0x4a, 0xa4, 0xb2, 0x54, 0xdb, 0x1c, 0x1e, 0x16, 0x33, 0x3f, - 0x0e, 0x8b, 0x0f, 0x82, 0x50, 0xee, 0xc5, 0x4d, 0xdb, 0x67, 0x1d, 0x67, 0x42, 0xd3, 0x6a, 0x4e, - 0xcf, 0x4e, 0xd4, 0x0e, 0x1c, 0x81, 0x7e, 0xcc, 0x43, 0x39, 0xb0, 0xb7, 0x9f, 0x3c, 0x56, 0xe0, - 0x5d, 0xaf, 0x83, 0x75, 0x85, 0xe6, 0x2e, 0xc5, 0x9a, 0x4b, 0xc9, 0xbc, 0x0d, 0x30, 0x51, 0x2e, - 0xcc, 0x6c, 0x89, 0x54, 0x96, 0x6b, 0x0b, 0x8a, 0xc6, 0x3d, 0xf6, 0x7e, 0x7d, 0xe1, 0xcb, 0xfb, - 0x22, 0x29, 0xff, 0x24, 0x70, 0x6d, 0xa2, 0x70, 0x03, 0x85, 0xcf, 0xc3, 0x48, 0x32, 0x6e, 0x54, - 0x61, 0x51, 0x01, 0x0a, 0x93, 0x94, 0x72, 0x95, 0xfc, 0xda, 0x1d, 0x7b, 0xa6, 0x75, 0xfb, 0xa4, - 0xc1, 0x94, 0x25, 0xb9, 0x69, 0x04, 0x90, 0x67, 0xaf, 0xba, 0x13, 0xbb, 0xd9, 0x7f, 0x6a, 0x17, - 0x34, 0x74, 0xe2, 0xf7, 0x1e, 0x5c, 0xee, 0x23, 0x17, 0x21, 0xeb, 0x9a, 0x39, 0x6d, 0x76, 0x35, - 0x25, 0x59, 0x39, 0xe1, 0xec, 0x59, 0x52, 0xe3, 0x1e, 0x15, 0xa7, 0x1d, 0xf8, 0x96, 0x03, 0x73, - 0x03, 0x77, 0xbd, 0x98, 0xca, 0xa9, 0x93, 0x4d, 0xc6, 0x5d, 0x46, 0xf1, 0xa2, 0x12, 0xfb, 0x4a, - 0xe0, 0x66, 0x2b, 0xd1, 0xd0, 0x98, 0x46, 0xd4, 0x88, 0x90, 0x37, 0x58, 0xf3, 0x25, 0xfa, 0xd2, - 0xcc, 0xea, 0x18, 0xb6, 0xe6, 0xc4, 0x30, 0x4f, 0xff, 0xe9, 0x0f, 0x75, 0xe4, 0x5b, 0x1a, 0xf1, - 0x51, 0x57, 0xf2, 0x41, 0xed, 0x85, 0xf2, 0xf2, 0xe6, 0x63, 0x71, 0xe7, 0x7c, 0x5e, 0x7a, 0xd4, - 0x11, 0xd8, 0x71, 0x24, 0x47, 0xb4, 0xab, 0x54, 0x22, 0x3f, 0x85, 0xbf, 0xe3, 0xf1, 0x00, 0x65, - 0x42, 0xe1, 0x16, 0x5a, 0x73, 0xe9, 0x0b, 0x03, 0x28, 0x9e, 0x21, 0xce, 0xb8, 0x0a, 0xb9, 0x36, - 0x0e, 0x74, 0xd7, 0x97, 0x5d, 0x75, 0x34, 0x1e, 0xc2, 0x62, 0xdf, 0xa3, 0x31, 0xea, 0x61, 0xca, - 0xaf, 0xdd, 0x9d, 0xd3, 0x8e, 0x19, 0x03, 0xed, 0x26, 0x17, 0xd7, 0xb3, 0xf7, 0x49, 0x9a, 0xfb, - 0xbb, 0x05, 0x28, 0xfc, 0xae, 0xe0, 0xd8, 0x02, 0x48, 0xb8, 0x31, 0x27, 0x11, 0xce, 0x28, 0xa6, - 0x6b, 0xe1, 0xfc, 0x65, 0x1e, 0xe9, 0x82, 0x98, 0xb3, 0xfa, 0xa2, 0xe7, 0xed, 0x3b, 0x81, 0xd2, - 0x0c, 0xda, 0x5d, 0xc6, 0x1b, 0x1e, 0xa5, 0x9a, 0x5a, 0xa4, 0xb3, 0xb0, 0x7d, 0x4e, 0xee, 0xa9, - 0xa7, 0x99, 0xb2, 0xaa, 0x94, 0x2a, 0x66, 0x71, 0x11, 0xf3, 0xb0, 0xda, 0xfa, 0x83, 0x80, 0xc2, - 0x6b, 0xb8, 0x75, 0xa6, 0xc0, 0xff, 0x3b, 0x13, 0xb5, 0xca, 0xf0, 0xb3, 0x95, 0x19, 0x8e, 0x2c, - 0xb2, 0x3f, 0xb2, 0xc8, 0xc1, 0xc8, 0x22, 0x9f, 0x46, 0x16, 0x79, 0x3b, 0xb6, 0x32, 0xfb, 0x63, - 0x2b, 0x73, 0x30, 0xb6, 0x32, 0xcf, 0x2f, 0x25, 0xbf, 0x86, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, - 0x8d, 0xf3, 0x2d, 0x5a, 0x2f, 0x06, 0x00, 0x00, + // 558 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x4f, 0x6b, 0x13, 0x41, + 0x14, 0xdf, 0x69, 0xd2, 0x6a, 0x27, 0x44, 0x64, 0xad, 0xb0, 0x44, 0xdd, 0x84, 0xa0, 0x10, 0x3c, + 0xec, 0x40, 0x0f, 0x22, 0x3d, 0x88, 0x09, 0xb5, 0xf4, 0x20, 0x34, 0xae, 0xd5, 0x83, 0x08, 0xcb, + 0x66, 0xf2, 0xb2, 0x5d, 0x33, 0xc9, 0x6c, 0x66, 0x66, 0x23, 0xb9, 0xfa, 0x09, 0x3c, 0x7a, 0x54, + 0xfc, 0x32, 0x39, 0xf6, 0xd8, 0x53, 0xd5, 0xe4, 0xe2, 0xd1, 0xb3, 0x17, 0x65, 0x76, 0xb7, 0x49, + 0x63, 0x13, 0x54, 0xf4, 0x10, 0x18, 0x32, 0xef, 0xfd, 0xfe, 0xbc, 0xdf, 0xbc, 0xc5, 0x55, 0x39, + 0x60, 0x84, 0xfa, 0xca, 0x67, 0x3c, 0x20, 0x6d, 0x90, 0x34, 0x6a, 0x91, 0x48, 0x84, 0xc3, 0x90, + 0x41, 0x00, 0x4e, 0x24, 0xb8, 0xe2, 0xe6, 0x75, 0xca, 0x69, 0x57, 0x70, 0x9f, 0x1e, 0x39, 0x72, + 0xc0, 0xf4, 0xaf, 0xe5, 0x4b, 0x28, 0x6d, 0x05, 0x3c, 0xe0, 0x49, 0x05, 0xd1, 0xa7, 0xb4, 0xb8, + 0xfa, 0x11, 0xe1, 0x2b, 0xcf, 0x24, 0x88, 0xe6, 0x19, 0x88, 0x34, 0x01, 0xe3, 0x58, 0x82, 0xf0, + 0x92, 0x02, 0x0b, 0x55, 0x50, 0x6d, 0xb3, 0xb1, 0x37, 0x3e, 0x2d, 0x1b, 0xdf, 0x4f, 0xcb, 0x0f, + 0x82, 0x50, 0x1d, 0xc5, 0x2d, 0x87, 0xf2, 0x1e, 0x99, 0xd1, 0xb4, 0x5b, 0xf3, 0x33, 0x89, 0xba, + 0x01, 0x91, 0x40, 0x63, 0x11, 0xaa, 0x91, 0xf3, 0xf4, 0xc9, 0x63, 0x0d, 0xde, 0xf7, 0x7b, 0xd0, + 0xd4, 0x68, 0xee, 0x66, 0x9c, 0x70, 0x69, 0x99, 0xb7, 0x31, 0x9e, 0x29, 0x97, 0xd6, 0x5a, 0x05, + 0xd5, 0x8a, 0x8d, 0xbc, 0xa6, 0x71, 0xcf, 0xfd, 0xbf, 0x93, 0xff, 0xfa, 0xbe, 0x8c, 0xaa, 0x3f, + 0x10, 0xbe, 0x36, 0x53, 0xb8, 0x0b, 0x92, 0x8a, 0x30, 0x52, 0x5c, 0x98, 0x75, 0xbc, 0xae, 0x01, + 0xa5, 0x85, 0x2a, 0xb9, 0x5a, 0x61, 0xfb, 0x8e, 0xb3, 0xd4, 0xba, 0xb3, 0x68, 0x30, 0x63, 0x49, + 0x3b, 0xcd, 0x00, 0x17, 0xf8, 0xeb, 0xfe, 0xcc, 0xee, 0xda, 0x7f, 0xb5, 0x8b, 0x13, 0xe8, 0xd4, + 0xef, 0x3d, 0x7c, 0x69, 0x08, 0x42, 0x86, 0xbc, 0x6f, 0xe5, 0x12, 0xb3, 0x37, 0x33, 0x92, 0xad, + 0x05, 0x67, 0xcf, 0xd3, 0x1a, 0xf7, 0xac, 0x38, 0x9b, 0xc0, 0x87, 0x3c, 0xb6, 0x76, 0xa1, 0xe3, + 0xc7, 0x4c, 0xcd, 0x9d, 0xec, 0x71, 0xe1, 0x72, 0x06, 0x26, 0x5d, 0x92, 0x58, 0xe3, 0xdf, 0xe5, + 0xef, 0x1b, 0x8b, 0x79, 0x15, 0x3b, 0x5c, 0x78, 0x3e, 0x63, 0x9e, 0xe0, 0x2c, 0x8b, 0xec, 0xf2, + 0xbe, 0xe1, 0x16, 0x3a, 0x5c, 0xd4, 0x19, 0xd3, 0x4a, 0xa4, 0xf9, 0x0d, 0xe1, 0x5b, 0xed, 0x54, + 0xa7, 0x37, 0x8f, 0xd1, 0x8b, 0x40, 0x78, 0xbc, 0xf5, 0x0a, 0xa8, 0xb2, 0x72, 0x49, 0x54, 0x07, + 0x2b, 0xa2, 0x5a, 0xe5, 0xf1, 0xe2, 0x45, 0x13, 0xc4, 0x41, 0x82, 0xf8, 0xa8, 0xaf, 0xc4, 0xa8, + 0xf1, 0x52, 0x4f, 0xf3, 0xcd, 0xa7, 0xf2, 0xe1, 0x9f, 0x79, 0x1e, 0x30, 0x22, 0xa1, 0x47, 0x94, + 0x00, 0x70, 0xea, 0x4c, 0x81, 0xb8, 0x80, 0x7f, 0xe8, 0x8b, 0x00, 0x54, 0x4a, 0xe1, 0x96, 0xda, + 0x2b, 0xe9, 0x4b, 0x23, 0x5c, 0xfe, 0x8d, 0x38, 0xf3, 0x2a, 0xce, 0x75, 0x61, 0x94, 0x24, 0x53, + 0x74, 0xf5, 0xd1, 0x7c, 0x88, 0xd7, 0x87, 0x3e, 0x8b, 0x21, 0x99, 0x62, 0x61, 0xfb, 0xee, 0x8a, + 0x71, 0x2c, 0x79, 0xf4, 0x6e, 0xda, 0xb8, 0xb3, 0x76, 0x1f, 0xa5, 0x6f, 0xa3, 0xb1, 0x81, 0xf3, + 0x3a, 0x91, 0xea, 0x3b, 0x84, 0x4b, 0xbf, 0x2a, 0x39, 0xb7, 0x2c, 0x0a, 0xdf, 0x58, 0x91, 0x8c, + 0xee, 0xce, 0x56, 0x88, 0xfc, 0x65, 0x2e, 0xd9, 0x32, 0x59, 0xcb, 0xe6, 0xa3, 0xef, 0x33, 0x89, + 0xb5, 0xf1, 0x17, 0xdb, 0x18, 0x4f, 0x6c, 0x74, 0x3c, 0xb1, 0xd1, 0xc9, 0xc4, 0x46, 0x9f, 0x27, + 0x36, 0x7a, 0x3b, 0xb5, 0x8d, 0xe3, 0xa9, 0x6d, 0x9c, 0x4c, 0x6d, 0xe3, 0xc5, 0x46, 0xfa, 0x35, + 0xfb, 0x19, 0x00, 0x00, 0xff, 0xff, 0x1a, 0x8f, 0xad, 0x2e, 0xe2, 0x04, 0x00, 0x00, } func (this *UserPrivileges) Equal(that interface{}) bool { @@ -316,7 +362,13 @@ func (this *DefaultPrivilegesForRole) Equal(that interface{}) bool { } else if this == nil { return false } - if this.UserProto != that1.UserProto { + if that1.Role == nil { + if this.Role != nil { + return false + } + } else if this.Role == nil { + return false + } else if !this.Role.Equal(that1.Role) { return false } if len(this.DefaultPrivilegesPerObject) != len(that1.DefaultPrivilegesPerObject) { @@ -331,6 +383,54 @@ func (this *DefaultPrivilegesForRole) Equal(that interface{}) bool { } return true } +func (this *DefaultPrivilegesForRole_UserProto) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DefaultPrivilegesForRole_UserProto) + if !ok { + that2, ok := that.(DefaultPrivilegesForRole_UserProto) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.UserProto != that1.UserProto { + return false + } + return true +} +func (this *DefaultPrivilegesForRole_ForAllRoles) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*DefaultPrivilegesForRole_ForAllRoles) + if !ok { + that2, ok := that.(DefaultPrivilegesForRole_ForAllRoles) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.ForAllRoles != that1.ForAllRoles { + return false + } + return true +} func (this *DefaultPrivilegeDescriptor) Equal(that interface{}) bool { if that == nil { return this == nil @@ -358,16 +458,6 @@ func (this *DefaultPrivilegeDescriptor) Equal(that interface{}) bool { return false } } - if len(this.DefaultPrivilegesForAllRoles) != len(that1.DefaultPrivilegesForAllRoles) { - return false - } - for i := range this.DefaultPrivilegesForAllRoles { - a := this.DefaultPrivilegesForAllRoles[i] - b := that1.DefaultPrivilegesForAllRoles[i] - if !(&a).Equal(&b) { - return false - } - } return true } func (m *UserPrivileges) Marshal() (dAtA []byte, err error) { @@ -490,9 +580,28 @@ func (m *DefaultPrivilegesForRole) MarshalToSizedBuffer(dAtA []byte) (int, error dAtA[i] = 0x8 i = encodeVarintPrivilege(dAtA, i, uint64(baseI-i)) i-- - dAtA[i] = 0x12 + dAtA[i] = 0x1a } } + if m.Role != nil { + { + size := m.Role.Size() + i -= size + if _, err := m.Role.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *DefaultPrivilegesForRole_UserProto) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *DefaultPrivilegesForRole_UserProto) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) i -= len(m.UserProto) copy(dAtA[i:], m.UserProto) i = encodeVarintPrivilege(dAtA, i, uint64(len(m.UserProto))) @@ -500,7 +609,23 @@ func (m *DefaultPrivilegesForRole) MarshalToSizedBuffer(dAtA []byte) (int, error dAtA[i] = 0xa return len(dAtA) - i, nil } +func (m *DefaultPrivilegesForRole_ForAllRoles) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} +func (m *DefaultPrivilegesForRole_ForAllRoles) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i-- + if m.ForAllRoles { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + return len(dAtA) - i, nil +} func (m *DefaultPrivilegeDescriptor) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -521,33 +646,6 @@ func (m *DefaultPrivilegeDescriptor) MarshalToSizedBuffer(dAtA []byte) (int, err _ = i var l int _ = l - if len(m.DefaultPrivilegesForAllRoles) > 0 { - keysForDefaultPrivilegesForAllRoles := make([]uint32, 0, len(m.DefaultPrivilegesForAllRoles)) - for k := range m.DefaultPrivilegesForAllRoles { - keysForDefaultPrivilegesForAllRoles = append(keysForDefaultPrivilegesForAllRoles, uint32(k)) - } - github_com_gogo_protobuf_sortkeys.Uint32s(keysForDefaultPrivilegesForAllRoles) - for iNdEx := len(keysForDefaultPrivilegesForAllRoles) - 1; iNdEx >= 0; iNdEx-- { - v := m.DefaultPrivilegesForAllRoles[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject(keysForDefaultPrivilegesForAllRoles[iNdEx])] - baseI := i - { - size, err := (&v).MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintPrivilege(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - i = encodeVarintPrivilege(dAtA, i, uint64(keysForDefaultPrivilegesForAllRoles[iNdEx])) - i-- - dAtA[i] = 0x8 - i = encodeVarintPrivilege(dAtA, i, uint64(baseI-i)) - i-- - dAtA[i] = 0x12 - } - } if len(m.DefaultPrivilegesPerRole) > 0 { for iNdEx := len(m.DefaultPrivilegesPerRole) - 1; iNdEx >= 0; iNdEx-- { { @@ -612,8 +710,9 @@ func (m *DefaultPrivilegesForRole) Size() (n int) { } var l int _ = l - l = len(m.UserProto) - n += 1 + l + sovPrivilege(uint64(l)) + if m.Role != nil { + n += m.Role.Size() + } if len(m.DefaultPrivilegesPerObject) > 0 { for k, v := range m.DefaultPrivilegesPerObject { _ = k @@ -626,6 +725,25 @@ func (m *DefaultPrivilegesForRole) Size() (n int) { return n } +func (m *DefaultPrivilegesForRole_UserProto) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.UserProto) + n += 1 + l + sovPrivilege(uint64(l)) + return n +} +func (m *DefaultPrivilegesForRole_ForAllRoles) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 2 + return n +} func (m *DefaultPrivilegeDescriptor) Size() (n int) { if m == nil { return 0 @@ -638,15 +756,6 @@ func (m *DefaultPrivilegeDescriptor) Size() (n int) { n += 1 + l + sovPrivilege(uint64(l)) } } - if len(m.DefaultPrivilegesForAllRoles) > 0 { - for k, v := range m.DefaultPrivilegesForAllRoles { - _ = k - _ = v - l = v.Size() - mapEntrySize := 1 + sovPrivilege(uint64(k)) + 1 + l + sovPrivilege(uint64(l)) - n += mapEntrySize + 1 + sovPrivilege(uint64(mapEntrySize)) - } - } return n } @@ -951,9 +1060,30 @@ func (m *DefaultPrivilegesForRole) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex]) + m.Role = &DefaultPrivilegesForRole_UserProto{github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])} iNdEx = postIndex case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ForAllRoles", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPrivilege + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.Role = &DefaultPrivilegesForRole_ForAllRoles{b} + case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field DefaultPrivilegesPerObject", wireType) } @@ -1152,121 +1282,6 @@ func (m *DefaultPrivilegeDescriptor) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DefaultPrivilegesForAllRoles", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPrivilege - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthPrivilege - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthPrivilege - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.DefaultPrivilegesForAllRoles == nil { - m.DefaultPrivilegesForAllRoles = make(map[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject]PrivilegeDescriptor) - } - var mapkey uint32 - mapvalue := &PrivilegeDescriptor{} - for iNdEx < postIndex { - entryPreIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPrivilege - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - if fieldNum == 1 { - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPrivilege - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapkey |= uint32(b&0x7F) << shift - if b < 0x80 { - break - } - } - } else if fieldNum == 2 { - var mapmsglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowPrivilege - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - mapmsglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if mapmsglen < 0 { - return ErrInvalidLengthPrivilege - } - postmsgIndex := iNdEx + mapmsglen - if postmsgIndex < 0 { - return ErrInvalidLengthPrivilege - } - if postmsgIndex > l { - return io.ErrUnexpectedEOF - } - mapvalue = &PrivilegeDescriptor{} - if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { - return err - } - iNdEx = postmsgIndex - } else { - iNdEx = entryPreIndex - skippy, err := skipPrivilege(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthPrivilege - } - if (iNdEx + skippy) > postIndex { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - m.DefaultPrivilegesForAllRoles[github_com_cockroachdb_cockroach_pkg_sql_sem_tree.AlterDefaultPrivilegesTargetObject(mapkey)] = *mapvalue - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPrivilege(dAtA[iNdEx:]) diff --git a/pkg/sql/catalog/descpb/privilege.proto b/pkg/sql/catalog/descpb/privilege.proto index a7f85d3164e9..441ec520aa14 100644 --- a/pkg/sql/catalog/descpb/privilege.proto +++ b/pkg/sql/catalog/descpb/privilege.proto @@ -48,9 +48,11 @@ message PrivilegeDescriptor { // the list of UserPrivileges for that object. message DefaultPrivilegesForRole { option (gogoproto.equal) = true; - optional string user_proto = 1 [(gogoproto.nullable) = false, - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"]; - map default_privileges_per_object = 2 [(gogoproto.nullable) = false, + oneof role { + string user_proto = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"]; + bool for_all_roles = 2; + } + map default_privileges_per_object = 3 [(gogoproto.nullable) = false, (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject"]; } @@ -60,6 +62,4 @@ message DefaultPrivilegesForRole { message DefaultPrivilegeDescriptor { option (gogoproto.equal) = true; repeated DefaultPrivilegesForRole default_privileges_per_role = 1 [(gogoproto.nullable) = false]; - map default_privileges_for_all_roles = 2 [(gogoproto.nullable) = false, - (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.AlterDefaultPrivilegesTargetObject"]; } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 25cef80f6414..0cee4f269916 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -4863,12 +4863,16 @@ CREATE TABLE crdb_internal.default_privileges ( for _, userPrivs := range privs.Users { privList := privilege.ListFromBitField(userPrivs.Privileges, privilegeObjectType) for _, priv := range privList { + role := tree.DNull + if !defaultPrivs.GetForAllRoles() { + role = tree.NewDString(defaultPrivs.GetUserProto().Decode().Normalized()) + } if err := addRow( tree.NewDString(descriptor.GetName()), // When the schema_name is NULL, that means the default // privileges are defined at the database level. tree.DNull, /* schema is currently always nil. See: #67376 */ - tree.NewDString(defaultPrivs.UserProto.Decode().Normalized()), + role, tree.NewDString(objectType.String()), tree.NewDString(userPrivs.User().Normalized()), tree.NewDString(priv.String()), diff --git a/pkg/sql/logictest/testdata/logic_test/bytes b/pkg/sql/logictest/testdata/logic_test/bytes index 29776ab6d458..05fa32cb2e02 100644 --- a/pkg/sql/logictest/testdata/logic_test/bytes +++ b/pkg/sql/logictest/testdata/logic_test/bytes @@ -153,4 +153,4 @@ PREPARE r1(bytes) AS SELECT descriptor::STRING FROM system.descriptor WHERE desc query T EXECUTE r1('abc') ---- -\022^\012\011defaultdb\0202\032\035\012\011\012\005admin\020\002\012\010\012\004root\020\002\022\004root\030\002"\000(\001@\000J\000Z(\022\010\010\001\022\004\022\000\030\000\022\010\010\002\022\004\022\000\030\000\022\010\010\003\022\004\022\000\030\000\022\010\010\004\022\004\022\000\030\000 +\0226\012\011defaultdb\0202\032\035\012\011\012\005admin\020\002\012\010\012\004root\020\002\022\004root\030\002"\000(\001@\000J\000Z\000 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog_pg_default_acl b/pkg/sql/logictest/testdata/logic_test/pg_catalog_pg_default_acl index f6847d505892..9ed0fb322882 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog_pg_default_acl +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog_pg_default_acl @@ -102,3 +102,33 @@ SELECT * FROM PG_CATALOG.PG_DEFAULT_ACL ---- oid defaclrole defaclnamespace defaclobjtype defaclacl 4149409857 1546506610 0 r {foo=/} + +# Check that entries show up for default privileges defined for all roles. +# The defaclrole oid should be 0. +statement ok +ALTER DEFAULT PRIVILEGES FOR ALL ROLES GRANT ALL ON TABLES TO foo, bar; +ALTER DEFAULT PRIVILEGES FOR ALL ROLES GRANT ALL ON TYPES TO foo, bar; +ALTER DEFAULT PRIVILEGES FOR ALL ROLES GRANT ALL ON SCHEMAS TO foo, bar; +ALTER DEFAULT PRIVILEGES FOR ALL ROLES GRANT ALL ON SEQUENCES TO foo, bar; + +query OOOTT colnames,rowsort +SELECT * FROM PG_CATALOG.PG_DEFAULT_ACL +---- +oid defaclrole defaclnamespace defaclobjtype defaclacl +4149409857 1546506610 0 r {foo=/} +2946850069 0 0 n {bar=CU/,foo=CU/} +2946850069 0 0 r {bar=Cadrw/,foo=Cadrw/} +2946850069 0 0 S {bar=Cadrw/,foo=Cadrw/} +2946850069 0 0 T {bar=U/,foo=U/} + +statement ok +ALTER DEFAULT PRIVILEGES FOR ALL ROLES REVOKE ALL ON TABLES FROM foo, bar; +ALTER DEFAULT PRIVILEGES FOR ALL ROLES REVOKE ALL ON TYPES FROM foo, bar; +ALTER DEFAULT PRIVILEGES FOR ALL ROLES REVOKE ALL ON SCHEMAS FROM foo, bar; +ALTER DEFAULT PRIVILEGES FOR ALL ROLES REVOKE ALL ON SEQUENCES FROM foo, bar; + +query OOOTT colnames,rowsort +SELECT * FROM PG_CATALOG.PG_DEFAULT_ACL +---- +oid defaclrole defaclnamespace defaclobjtype defaclacl +4149409857 1546506610 0 r {foo=/} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index 842e3df21718..d932e476de46 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic @@ -18,7 +18,7 @@ WHERE message NOT LIKE '%Z/%' AND operation != 'dist sender send' ---- batch flow coordinator CPut /NamespaceTable/30/1/0/0/"t"/4/1 -> 53 -batch flow coordinator CPut /Table/3/1/53/2/1 -> database: version:1 privileges: users: owner_proto:"root" version:2 > state:PUBLIC offline_reason:"" default_privileges: > default_privileges_for_all_roles: > default_privileges_for_all_roles: > default_privileges_for_all_roles: > > > +batch flow coordinator CPut /Table/3/1/53/2/1 -> database: version:1 privileges: users: owner_proto:"root" version:2 > state:PUBLIC offline_reason:"" default_privileges:<> > batch flow coordinator CPut /NamespaceTable/30/1/53/0/"public"/4/1 -> 29 exec stmt rows affected: 0 diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index fa8c7518a808..34962dd34e26 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -1175,14 +1175,22 @@ https://www.postgresql.org/docs/13/catalog-pg-default-acl.html`, // schemas are supported. // See: https://github.com/cockroachdb/cockroach/issues/67376. schemaName := "" + // If ForAllRoles is specified, we use an empty string as the normalized + // role name to create the row hash. + normalizedName := "" + roleOid := oidZero + if !defaultPrivs.GetForAllRoles() { + roleOid = h.UserOid(defaultPrivs.GetUserProto().Decode()) + normalizedName = defaultPrivs.GetUserProto().Decode().Normalized() + } rowOid := h.DBSchemaRoleOid( dbContext.GetID(), schemaName, - defaultPrivs.UserProto.Decode().Normalized(), + normalizedName, ) if err := addRow( - rowOid, // row identifier oid - h.UserOid(defaultPrivs.UserProto.Decode()), // defaclrole oid + rowOid, // row identifier oid + roleOid, // defaclrole oid oidZero, // defaclnamespace oid tree.NewDString(c), // defaclobjtype char arr, // defaclacl aclitem[] From 4740eb801f49d75156638c95a879828428e324dc Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Fri, 6 Aug 2021 01:08:45 -0400 Subject: [PATCH 6/8] *: update to lib/pq v1.10.2 This gets us off the cockroachdb/pq fork. This revealed a bug in some of the tests: they were not calling rows.Close() after querying the database. The new version of lib/pq is more strict about requiring this. Additionally, logic tests were not checking `NextResultSet()`, which is a bug if the test executes multiple queries at once. Release note: none --- DEPS.bzl | 5 +- go.mod | 5 +- go.sum | 9 +- pkg/cli/doctor.go | 12 ++- pkg/sql/logictest/logic.go | 137 ++++++++++++++++--------------- pkg/sql/temporary_schema_test.go | 8 +- vendor | 2 +- 7 files changed, 96 insertions(+), 82 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 774eb2f78eb5..9f2d3d48afbf 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2603,9 +2603,8 @@ def go_deps(): name = "com_github_lib_pq", build_file_proto_mode = "disable_global", importpath = "github.com/lib/pq", - replace = "github.com/cockroachdb/pq", - sum = "h1:xTc0ViFhuelzQZAYQOxMR2J5QDO9/C+0L0fkPXIcoMI=", - version = "v0.0.0-20210517091544-990dd3347596", + sum = "h1:AqzbZs4ZoCBp+GtejcpCpcxM3zlSMx29dXbUSeVtJb8=", + version = "v1.10.2", ) go_repository( name = "com_github_lib_pq_auth_kerberos", diff --git a/go.mod b/go.mod index 22721dcd59f4..3d74d19c96b5 100644 --- a/go.mod +++ b/go.mod @@ -103,7 +103,7 @@ require ( github.com/kr/pretty v0.2.1 github.com/kr/text v0.2.0 github.com/leanovate/gopter v0.2.5-0.20190402064358-634a59d12406 - github.com/lib/pq v1.8.0 + github.com/lib/pq v1.10.2 github.com/lib/pq/auth/kerberos v0.0.0-20200720160335-984a6aa1ca46 github.com/lightstep/lightstep-tracer-go v0.24.0 github.com/linkedin/goavro/v2 v2.10.0 @@ -192,9 +192,6 @@ replace gopkg.in/yaml.v2 => github.com/cockroachdb/yaml v0.0.0-20180705215940-0e replace github.com/knz/go-libedit => github.com/otan-cockroach/go-libedit v1.10.2-0.20201030151939-7cced08450e7 -// Temporary workaround for #65320. -replace github.com/lib/pq => github.com/cockroachdb/pq v0.0.0-20210517091544-990dd3347596 - // At the time of writing (i.e. as of this version below) the `etcd` repo is in the process of properly introducing // modules, and as part of that uses an unsatisfiable version for this dependency (v3.0.0-00010101000000-000000000000). // We just force it to the same SHA as the `go.etcd.io/etcd/raft/v3` module (they live in the same VCS root). diff --git a/go.sum b/go.sum index c4be88707a8a..9b25f939fe83 100644 --- a/go.sum +++ b/go.sum @@ -267,8 +267,6 @@ github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOi github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5 h1:Igd6YmtOZ77EgLAIaE9+mHl7+sAKaZ5m4iMI0Dz/J2A= github.com/cockroachdb/pebble v0.0.0-20210719141320-8c3bd06debb5/go.mod h1:JXfQr3d+XO4bL1pxGwKKo09xylQSdZ/mpZ9b2wfVcPs= -github.com/cockroachdb/pq v0.0.0-20210517091544-990dd3347596 h1:xTc0ViFhuelzQZAYQOxMR2J5QDO9/C+0L0fkPXIcoMI= -github.com/cockroachdb/pq v0.0.0-20210517091544-990dd3347596/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.0/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ= @@ -944,6 +942,13 @@ github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvf github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.5-0.20190402064358-634a59d12406 h1:+OUpk+IVvmKU0jivOVFGtOzA6U5AWFs8HE4DRzWLOUE= github.com/leanovate/gopter v0.2.5-0.20190402064358-634a59d12406/go.mod h1:gNcbPWNEWRe4lm+bycKqxUYoH5uoVje5SkOJ3uoLer8= +github.com/lib/pq v0.0.0-20180327071824-d34b9ff171c2/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.0.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.1.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.2.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/lib/pq v1.8.0/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= +github.com/lib/pq v1.10.2 h1:AqzbZs4ZoCBp+GtejcpCpcxM3zlSMx29dXbUSeVtJb8= +github.com/lib/pq v1.10.2/go.mod h1:AlVN5x4E4T544tWzH6hKfbfQvm3HdbOxrmggDNAPY9o= github.com/lib/pq/auth/kerberos v0.0.0-20200720160335-984a6aa1ca46 h1:q7hY+WNJTcSqJNGwJzXZYL++nWBaoKlKdgZOyY6jxz4= github.com/lib/pq/auth/kerberos v0.0.0-20200720160335-984a6aa1ca46/go.mod h1:jydegJvs5JvVcuFD/YAT8JRmRVeOoRhtnGEgRnAoPpE= github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20200305213919-a88bf8de3718 h1:lrdADj7ifyBpqGJ+cT4vE5ztUoAF87uUf76+epwPViY= diff --git a/pkg/cli/doctor.go b/pkg/cli/doctor.go index fdfba6d396ed..e7802c0a33b7 100644 --- a/pkg/cli/doctor.go +++ b/pkg/cli/doctor.go @@ -190,8 +190,8 @@ func fromCluster( stmt := ` SELECT id, descriptor, crdb_internal_mvcc_timestamp AS mod_time_logical FROM system.descriptor ORDER BY id` - checkColumnExistsStmt := "SELECT crdb_internal_mvcc_timestamp" - _, err := sqlConn.Query(maybePrint(checkColumnExistsStmt), nil) + checkColumnExistsStmt := "SELECT crdb_internal_mvcc_timestamp FROM system.descriptor LIMIT 1" + _, err := sqlConn.QueryRow(maybePrint(checkColumnExistsStmt), nil) // On versions before 20.2, the system.descriptor won't have the builtin // crdb_internal_mvcc_timestamp. If we can't find it, use NULL instead. if pqErr := (*pq.Error)(nil); errors.As(err, &pqErr) { @@ -200,6 +200,8 @@ FROM system.descriptor ORDER BY id` SELECT id, descriptor, NULL AS mod_time_logical FROM system.descriptor ORDER BY id` } + } else if err != nil { + return nil, nil, nil, err } descTable = make([]doctor.DescriptorTableRow, 0) @@ -238,8 +240,8 @@ FROM system.descriptor ORDER BY id` stmt = `SELECT "parentID", "parentSchemaID", name, id FROM system.namespace` - checkColumnExistsStmt = `SELECT "parentSchemaID" FROM system.namespace LIMIT 0` - _, err = sqlConn.Query(maybePrint(checkColumnExistsStmt), nil) + checkColumnExistsStmt = `SELECT "parentSchemaID" FROM system.namespace LIMIT 1` + _, err = sqlConn.QueryRow(maybePrint(checkColumnExistsStmt), nil) // On versions before 20.1, table system.namespace does not have this column. // In that case the ParentSchemaID for tables is 29 and for databases is 0. if pqErr := (*pq.Error)(nil); errors.As(err, &pqErr) { @@ -248,6 +250,8 @@ FROM system.descriptor ORDER BY id` SELECT "parentID", CASE WHEN "parentID" = 0 THEN 0 ELSE 29 END AS "parentSchemaID", name, id FROM system.namespace` } + } else if err != nil { + return nil, nil, nil, err } namespaceTable = make([]doctor.NamespaceTableRow, 0) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index b9f267dd0649..c6f4759b0ad7 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -2669,83 +2669,88 @@ func (t *logicTest) execQuery(query logicQuery) error { if query.colNames { actualResultsRaw = append(actualResultsRaw, cols...) } - for rows.Next() { - if err := rows.Scan(vals...); err != nil { - return err - } - for i, v := range vals { - if val := *v.(*interface{}); val != nil { - valT := reflect.TypeOf(val).Kind() - colT := query.colTypes[i] - switch colT { - case 'T': - if valT != reflect.String && valT != reflect.Slice && valT != reflect.Struct { - return fmt.Errorf("%s: expected text value for column %d, but found %T: %#v", - query.pos, i, val, val, - ) - } - case 'I': - if valT != reflect.Int64 { - if *flexTypes && (valT == reflect.Float64 || valT == reflect.Slice) { - t.signalIgnoredError( - fmt.Errorf("result type mismatch: expected I, got %T", val), query.pos, query.sql, + for nextResultSet := true; nextResultSet; nextResultSet = rows.NextResultSet() { + for rows.Next() { + if err := rows.Scan(vals...); err != nil { + return err + } + for i, v := range vals { + if val := *v.(*interface{}); val != nil { + valT := reflect.TypeOf(val).Kind() + colT := query.colTypes[i] + switch colT { + case 'T': + if valT != reflect.String && valT != reflect.Slice && valT != reflect.Struct { + return fmt.Errorf("%s: expected text value for column %d, but found %T: %#v", + query.pos, i, val, val, ) - return nil } - return fmt.Errorf("%s: expected int value for column %d, but found %T: %#v", - query.pos, i, val, val, - ) - } - case 'F', 'R': - if valT != reflect.Float64 && valT != reflect.Slice { - if *flexTypes && (valT == reflect.Int64) { - t.signalIgnoredError( - fmt.Errorf("result type mismatch: expected F or R, got %T", val), query.pos, query.sql, + case 'I': + if valT != reflect.Int64 { + if *flexTypes && (valT == reflect.Float64 || valT == reflect.Slice) { + t.signalIgnoredError( + fmt.Errorf("result type mismatch: expected I, got %T", val), query.pos, query.sql, + ) + return nil + } + return fmt.Errorf("%s: expected int value for column %d, but found %T: %#v", + query.pos, i, val, val, ) - return nil } - return fmt.Errorf("%s: expected float/decimal value for column %d, but found %T: %#v", - query.pos, i, val, val, - ) - } - case 'B': - if valT != reflect.Bool { - return fmt.Errorf("%s: expected boolean value for column %d, but found %T: %#v", - query.pos, i, val, val, - ) - } - case 'O': - if valT != reflect.Slice { - return fmt.Errorf("%s: expected oid value for column %d, but found %T: %#v", - query.pos, i, val, val, + case 'F', 'R': + if valT != reflect.Float64 && valT != reflect.Slice { + if *flexTypes && (valT == reflect.Int64) { + t.signalIgnoredError( + fmt.Errorf("result type mismatch: expected F or R, got %T", val), query.pos, query.sql, + ) + return nil + } + return fmt.Errorf("%s: expected float/decimal value for column %d, but found %T: %#v", + query.pos, i, val, val, + ) + } + case 'B': + if valT != reflect.Bool { + return fmt.Errorf("%s: expected boolean value for column %d, but found %T: %#v", + query.pos, i, val, val, + ) + } + case 'O': + if valT != reflect.Slice { + return fmt.Errorf("%s: expected oid value for column %d, but found %T: %#v", + query.pos, i, val, val, + ) + } + default: + return fmt.Errorf("%s: unknown type in type string: %c in %s", + query.pos, colT, query.colTypes, ) } - default: - return fmt.Errorf("%s: unknown type in type string: %c in %s", - query.pos, colT, query.colTypes, - ) - } - if byteArray, ok := val.([]byte); ok { - // The postgres wire protocol does not distinguish between - // strings and byte arrays, but our tests do. In order to do - // The Right Thingâ„¢, we replace byte arrays which are valid - // UTF-8 with strings. This allows byte arrays which are not - // valid UTF-8 to print as a list of bytes (e.g. `[124 107]`) - // while printing valid strings naturally. - if str := string(byteArray); utf8.ValidString(str) { - val = str + if byteArray, ok := val.([]byte); ok { + // The postgres wire protocol does not distinguish between + // strings and byte arrays, but our tests do. In order to do + // The Right Thingâ„¢, we replace byte arrays which are valid + // UTF-8 with strings. This allows byte arrays which are not + // valid UTF-8 to print as a list of bytes (e.g. `[124 107]`) + // while printing valid strings naturally. + if str := string(byteArray); utf8.ValidString(str) { + val = str + } } + // Empty strings are rendered as "·" (middle dot) + if val == "" { + val = "·" + } + actualResultsRaw = append(actualResultsRaw, fmt.Sprint(val)) + } else { + actualResultsRaw = append(actualResultsRaw, "NULL") } - // Empty strings are rendered as "·" (middle dot) - if val == "" { - val = "·" - } - actualResultsRaw = append(actualResultsRaw, fmt.Sprint(val)) - } else { - actualResultsRaw = append(actualResultsRaw, "NULL") } } + if err := rows.Err(); err != nil { + return err + } } if err := rows.Err(); err != nil { return err diff --git a/pkg/sql/temporary_schema_test.go b/pkg/sql/temporary_schema_test.go index 6d1616bfb7b9..b1e6c2d31eb7 100644 --- a/pkg/sql/temporary_schema_test.go +++ b/pkg/sql/temporary_schema_test.go @@ -82,8 +82,10 @@ INSERT INTO perm_table VALUES (DEFAULT, 1); } for _, name := range selectableTempNames { // Check tables are accessible. - _, err = conn.QueryContext(ctx, fmt.Sprintf("SELECT * FROM %s.%s", tempSchemaName, name)) + var rows *gosql.Rows + rows, err = conn.QueryContext(ctx, fmt.Sprintf("SELECT * FROM %s.%s", tempSchemaName, name)) require.NoError(t, err) + require.NoError(t, rows.Close()) } require.NoError( @@ -112,8 +114,10 @@ INSERT INTO perm_table VALUES (DEFAULT, 1); ensureTemporaryObjectsAreDeleted(ctx, t, conn, tempSchemaName, tempNames) // Check perm_table performs correctly, and has the right schema. - _, err = db.Query("SELECT * FROM perm_table") + var rows *gosql.Rows + rows, err = db.Query("SELECT * FROM perm_table") require.NoError(t, err) + require.NoError(t, rows.Close()) var colDefault gosql.NullString err = db.QueryRow( diff --git a/vendor b/vendor index e1e6c8351344..173d3bda7a0a 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit e1e6c835134419ded2a45b6e528713ebd8fe6830 +Subproject commit 173d3bda7a0a24810b236e07d67e3e8443454329 From 9697387ebc55383a7b4684cf531b25c0d557b756 Mon Sep 17 00:00:00 2001 From: Ricky Stewart Date: Thu, 5 Aug 2021 16:28:01 -0500 Subject: [PATCH 7/8] dev: rip recording logic out of `dev` datadriven tests This feature isn't used by anyone that currently works on `dev`, and manually updating the recording files seems to be the most convenient way to work on these tests. In the future we may consider migrating to a proper mocking framework for `dev`, but for now this structure is fine. Also rename recorder -> recording for accuracy. Release note: None --- pkg/cmd/dev/BUILD.bazel | 2 +- pkg/cmd/dev/bench.go | 2 +- pkg/cmd/dev/build.go | 2 +- pkg/cmd/dev/builder.go | 2 +- pkg/cmd/dev/datadriven_test.go | 93 +++---------- pkg/cmd/dev/io/exec/BUILD.bazel | 5 +- pkg/cmd/dev/io/exec/exec.go | 84 +++--------- pkg/cmd/dev/io/os/BUILD.bazel | 2 +- pkg/cmd/dev/io/os/os.go | 127 +++--------------- .../dev/{recorder => recording}/BUILD.bazel | 6 +- .../dev/{recorder => recording}/operation.go | 2 +- .../recorder.go => recording/recording.go} | 76 +++-------- .../dev/{recorder => recording}/scanner.go | 2 +- pkg/cmd/dev/test.go | 2 +- 14 files changed, 84 insertions(+), 323 deletions(-) rename pkg/cmd/dev/{recorder => recording}/BUILD.bazel (80%) rename pkg/cmd/dev/{recorder => recording}/operation.go (98%) rename pkg/cmd/dev/{recorder/recorder.go => recording/recording.go} (69%) rename pkg/cmd/dev/{recorder => recording}/scanner.go (98%) diff --git a/pkg/cmd/dev/BUILD.bazel b/pkg/cmd/dev/BUILD.bazel index 55c29bd13a52..877edfc6a5c1 100644 --- a/pkg/cmd/dev/BUILD.bazel +++ b/pkg/cmd/dev/BUILD.bazel @@ -38,7 +38,7 @@ go_test( deps = [ "//pkg/cmd/dev/io/exec", "//pkg/cmd/dev/io/os", - "//pkg/cmd/dev/recorder", + "//pkg/cmd/dev/recording", "//pkg/testutils", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_stretchr_testify//require", diff --git a/pkg/cmd/dev/bench.go b/pkg/cmd/dev/bench.go index 27a75aa6847f..84696f328236 100644 --- a/pkg/cmd/dev/bench.go +++ b/pkg/cmd/dev/bench.go @@ -104,7 +104,7 @@ func (d *dev) bench(cmd *cobra.Command, pkgs []string) error { if timeout > 0 { args = append(args, fmt.Sprintf("-test.timeout=%s", timeout.String())) } - err := d.exec.CommandContextNoRecord(ctx, "bazel", args...) + err := d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...) if err != nil { return err } diff --git a/pkg/cmd/dev/build.go b/pkg/cmd/dev/build.go index 42ab7f40ffe7..df729c7809ba 100644 --- a/pkg/cmd/dev/build.go +++ b/pkg/cmd/dev/build.go @@ -81,7 +81,7 @@ func (d *dev) build(cmd *cobra.Command, targets []string) error { if cross == "" { args = append(args, getConfigFlags()...) - if err := d.exec.CommandContextNoRecord(ctx, "bazel", args...); err != nil { + if err := d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...); err != nil { return err } return d.symlinkBinaries(ctx, fullTargets) diff --git a/pkg/cmd/dev/builder.go b/pkg/cmd/dev/builder.go index 237c8271d33c..9c22df297292 100644 --- a/pkg/cmd/dev/builder.go +++ b/pkg/cmd/dev/builder.go @@ -43,7 +43,7 @@ func (d *dev) builder(cmd *cobra.Command, _ []string) error { if err != nil { return err } - return d.exec.CommandContextNoRecord(ctx, "docker", args...) + return d.exec.CommandContextInheritingStdStreams(ctx, "docker", args...) } func (d *dev) ensureDockerVolume(ctx context.Context, volume string) error { diff --git a/pkg/cmd/dev/datadriven_test.go b/pkg/cmd/dev/datadriven_test.go index f78abb2cfd40..30e6f27c3e03 100644 --- a/pkg/cmd/dev/datadriven_test.go +++ b/pkg/cmd/dev/datadriven_test.go @@ -12,9 +12,7 @@ package main import ( "bytes" - "flag" "fmt" - "go/build" "io" "io/ioutil" "log" @@ -25,31 +23,19 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/dev/io/exec" "github.com/cockroachdb/cockroach/pkg/cmd/dev/io/os" - "github.com/cockroachdb/cockroach/pkg/cmd/dev/recorder" + "github.com/cockroachdb/cockroach/pkg/cmd/dev/recording" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" ) -var ( - recordFlag = flag.Bool( - "record", false, - "ignore existing recordings and rewrite them with results from an actual execution (see -from-checkout)", - ) - - fromCheckoutFlag = flag.String( - "from-checkout", crdbpath(), - "cockroach/cockroachdb checkout to generate recordings from", - ) -) - func init() { isTesting = true } -// TestDatadriven makes use of datadriven and recorder to capture all operations -// executed by individual `dev` invocations. The testcases are defined under -// testdata/*, where each test files corresponds to a recording capture found in +// TestDatadriven makes use of datadriven to play back all operations executed +// by individual `dev` invocations. The testcases are defined under testdata/*, +// where each test files corresponds to a recording capture found in // testdata/recording/*. // // Datadriven divvies up these files as subtests, so individual "files" are @@ -57,24 +43,10 @@ func init() { // // go test -run TestDatadriven/ // -// To update the test file with new capture data, try: -// -// go test -run TestDatadriven/ -rewrite -record \ -// [-from-checkout=] -// -// The -rewrite flag rewrites what is found under testdata/, while the -// -record flag rewrites what is found under testdata/recording/. // Recordings are used to mock out "system" behavior. During these test runs -// (unless -record is specified), attempts to shell out to `bazel` are -// intercepted and responses are constructed using recorded data. The same is -// true for attempts to run os operations (like creating/removing/symlinking -// filepaths). -// -// In summary: the traces for all operations attempted as part of test run are -// captured within testdata/ and the mocked out responses for each -// "external" command can be found under testadata/recording/. The former -// is updated by specifying -rewrite, the latter is updated by specifying -// -record (and optionally -from-checkout). +// (unless -record is specified), attempts to shell out to `bazel` or perform +// other OS operations are intercepted and responses are constructed using +// recorded data. func TestDatadriven(t *testing.T) { verbose := testing.Verbose() testdata := testutils.TestDataPath(t) @@ -88,7 +60,6 @@ func TestDatadriven(t *testing.T) { // We'll match against printed logs for datadriven. var logger io.ReadWriter = bytes.NewBufferString("") - var recording io.ReadWriter var exopts []exec.Option var osopts []os.Option @@ -100,25 +71,15 @@ func TestDatadriven(t *testing.T) { exopts = append(exopts, exec.WithStdOutErr(ioutil.Discard, ioutil.Discard)) } - if *recordFlag { - recording = bytes.NewBufferString("") - - r := recorder.New(recorder.WithRecordingTo(recording)) // the thing to record into - exopts = append(exopts, exec.WithWorkingDir(*fromCheckoutFlag)) // the path to run dev from - osopts = append(osopts, os.WithWorkingDir(*fromCheckoutFlag)) // the path to run dev from - exopts = append(exopts, exec.WithRecorder(r)) - osopts = append(osopts, os.WithRecorder(r)) - } else { - frecording, err := stdos.OpenFile(recordingPath, stdos.O_RDONLY, 0600) - require.NoError(t, err) - defer func() { - require.NoError(t, frecording.Close()) - }() - - r := recorder.New(recorder.WithReplayFrom(frecording, recordingPath)) // the recording we're playing back from - exopts = append(exopts, exec.WithRecorder(r)) - osopts = append(osopts, os.WithRecorder(r)) - } + frecording, err := stdos.OpenFile(recordingPath, stdos.O_RDONLY, 0600) + require.NoError(t, err) + defer func() { + require.NoError(t, frecording.Close()) + }() + + r := recording.WithReplayFrom(frecording, recordingPath) + exopts = append(exopts, exec.WithRecording(r)) + osopts = append(osopts, os.WithRecording(r)) devExec := exec.New(exopts...) devOS := os.New(osopts...) @@ -154,27 +115,5 @@ func TestDatadriven(t *testing.T) { return fmt.Sprintf("unknown command: %s", d.Cmd) } }) - - if *recordFlag { - recording, err := ioutil.ReadAll(recording) - require.NoError(t, err) - - frecording, err := stdos.OpenFile(recordingPath, stdos.O_CREATE|stdos.O_WRONLY|stdos.O_TRUNC|stdos.O_SYNC, 0600) - require.NoError(t, err) - defer func() { - require.NoError(t, frecording.Close()) - }() - - _, err = frecording.Write(recording) - require.NoError(t, err) - } }) } - -func crdbpath() string { - gopath := stdos.Getenv("GOPATH") - if gopath == "" { - gopath = build.Default.GOPATH - } - return filepath.Join(gopath, "src", "github.com", "cockroachdb", "cockroach") -} diff --git a/pkg/cmd/dev/io/exec/BUILD.bazel b/pkg/cmd/dev/io/exec/BUILD.bazel index 112a69035ea1..9bcdd5733fcf 100644 --- a/pkg/cmd/dev/io/exec/BUILD.bazel +++ b/pkg/cmd/dev/io/exec/BUILD.bazel @@ -5,8 +5,5 @@ go_library( srcs = ["exec.go"], importpath = "github.com/cockroachdb/cockroach/pkg/cmd/dev/io/exec", visibility = ["//visibility:public"], - deps = [ - "//pkg/cmd/dev/recorder", - "@com_github_cockroachdb_errors//:errors", - ], + deps = ["//pkg/cmd/dev/recording"], ) diff --git a/pkg/cmd/dev/io/exec/exec.go b/pkg/cmd/dev/io/exec/exec.go index e0c33561a9f9..eabf2b8206dd 100644 --- a/pkg/cmd/dev/io/exec/exec.go +++ b/pkg/cmd/dev/io/exec/exec.go @@ -21,26 +21,16 @@ import ( "os/exec" "strings" - "github.com/cockroachdb/cockroach/pkg/cmd/dev/recorder" - "github.com/cockroachdb/errors" + "github.com/cockroachdb/cockroach/pkg/cmd/dev/recording" ) -// Exec is a convenience wrapper around the stdlib os/exec package. It lets us: -// -// (a) mock all instances where we shell out, for tests, and -// (b) capture all instances of shelling out that take place during execution -// -// We achieve (a) by embedding a Recorder, and either replaying from it if -// configured to do so, or "doing the real thing" and recording the fact into -// the Recorder for future playback. -// -// For (b), each operation is logged (if configured to do so). These messages -// can be captured by the caller and compared against what is expected. +// Exec is a convenience wrapper around the stdlib os/exec package. It lets us +// mock all instances where we shell out for tests. type Exec struct { dir string logger *log.Logger stdout, stderr io.Writer - *recorder.Recorder + *recording.Recording } // New returns a new Exec with the given options. @@ -83,17 +73,10 @@ func WithStdOutErr(stdout, stderr io.Writer) func(e *Exec) { } } -// WithRecorder configures Exec to use the provided recorder. -func WithRecorder(r *recorder.Recorder) func(e *Exec) { - return func(e *Exec) { - e.Recorder = r - } -} - -// WithWorkingDir configures Exec to use the provided working directory. -func WithWorkingDir(dir string) func(e *Exec) { +// WithRecording configures Exec to use the provided recording. +func WithRecording(r *recording.Recording) func(e *Exec) { return func(e *Exec) { - e.dir = dir + e.Recording = r } } @@ -120,11 +103,11 @@ func (e *Exec) CommandContextWithInput( return e.commandContextImpl(ctx, r, false, name, args...) } -// CommandContextNoRecord is like CommandContext, but doesn't capture stdout. -// To be used when we want to run a subprocess but deliberately want to pass -// stdin, stdout, and stderr right to the terminal. Note that this can't be -// used for recording and the process will panic if you try to record with it. -func (e *Exec) CommandContextNoRecord(ctx context.Context, name string, args ...string) error { +// CommandContextInheritingStdStreams is like CommandContext, but stdin, +// stdout, and stderr are passed directly to the terminal. +func (e *Exec) CommandContextInheritingStdStreams( + ctx context.Context, name string, args ...string, +) error { var command string if len(args) > 0 { command = fmt.Sprintf("%s %s", name, strings.Join(args, " ")) @@ -133,11 +116,7 @@ func (e *Exec) CommandContextNoRecord(ctx context.Context, name string, args ... } e.logger.Print(command) - if e.Recorder != nil && e.Recorder.Recording() { - return errors.New("Can't call CommandContextNoRecord while recording") - } - - if e.Recorder == nil { + if e.Recording == nil { // Do the real thing. cmd := exec.CommandContext(ctx, name, args...) cmd.Stdin = os.Stdin @@ -164,23 +143,16 @@ func (e *Exec) LookPath(path string) (string, error) { command := fmt.Sprintf("which %s", path) e.logger.Print(command) - var fullPath string - if e.Recorder == nil || e.Recorder.Recording() { + if e.Recording == nil { // Do the real thing. var err error - fullPath, err = exec.LookPath(path) + fullPath, err := exec.LookPath(path) if err != nil { return "", err } - } - - if e.Recorder == nil { return fullPath, nil } - if e.Recording() { - return fullPath, e.record(command, fullPath) - } ret, err := e.replay(command) return ret, err } @@ -197,7 +169,7 @@ func (e *Exec) commandContextImpl( e.logger.Print(command) var buffer bytes.Buffer - if e.Recorder == nil || e.Recorder.Recording() { + if e.Recording == nil { // Do the real thing. cmd := exec.CommandContext(ctx, name, args...) if silent { @@ -218,16 +190,6 @@ func (e *Exec) commandContextImpl( if err := cmd.Wait(); err != nil { return nil, err } - } - - if e.Recorder == nil { - return buffer.Bytes(), nil - } - - if e.Recording() { - if err := e.record(command, buffer.String()); err != nil { - return nil, err - } return buffer.Bytes(), nil } @@ -240,9 +202,9 @@ func (e *Exec) commandContextImpl( } // replay replays the specified command, erroring out if it's mismatched with -// what the recorder plays back next. It returns the recorded output. +// what the recording plays back next. It returns the recorded output. func (e *Exec) replay(command string) (output string, err error) { - found, err := e.Recorder.Next(func(op recorder.Operation) error { + found, err := e.Recording.Next(func(op recording.Operation) error { if op.Command != command { return fmt.Errorf("expected %q, got %q", op.Command, command) } @@ -257,13 +219,3 @@ func (e *Exec) replay(command string) (output string, err error) { } return output, nil } - -// record records the specified command with the corresponding output. -func (e *Exec) record(command, output string) error { - op := recorder.Operation{ - Command: command, - Output: output, - } - - return e.Record(op) -} diff --git a/pkg/cmd/dev/io/os/BUILD.bazel b/pkg/cmd/dev/io/os/BUILD.bazel index 89fc03d26a80..9c917fe7a737 100644 --- a/pkg/cmd/dev/io/os/BUILD.bazel +++ b/pkg/cmd/dev/io/os/BUILD.bazel @@ -6,7 +6,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cmd/dev/io/os", visibility = ["//visibility:public"], deps = [ - "//pkg/cmd/dev/recorder", + "//pkg/cmd/dev/recording", "@com_github_cockroachdb_errors//oserror", ], ) diff --git a/pkg/cmd/dev/io/os/os.go b/pkg/cmd/dev/io/os/os.go index d683ed74cdc7..853fb93ae969 100644 --- a/pkg/cmd/dev/io/os/os.go +++ b/pkg/cmd/dev/io/os/os.go @@ -16,25 +16,15 @@ import ( "log" "os" - "github.com/cockroachdb/cockroach/pkg/cmd/dev/recorder" + "github.com/cockroachdb/cockroach/pkg/cmd/dev/recording" "github.com/cockroachdb/errors/oserror" ) -// OS is a convenience wrapper around the stdlib os package. It lets us: -// -// (a) mock operating system calls in tests, and -// (b) capture the set of calls that take place during execution -// -// We achieve (a) by embedding a Recorder, and either replaying from it if -// configured to do so, or "doing the real thing" and recording the fact into -// the Recorder for future playback. -// -// For (b), each operation is logged (if configured to do so). These messages -// can be captured by the caller and compared against what is expected. +// OS is a convenience wrapper around the stdlib os package. It lets us +// mock operating system calls in tests. type OS struct { - dir string logger *log.Logger - *recorder.Recorder + *recording.Recording } // New constructs a new OS handle, configured with the provided options. @@ -68,17 +58,10 @@ func WithLogger(logger *log.Logger) func(o *OS) { } } -// WithRecorder configures OS to use the provided recorder. -func WithRecorder(r *recorder.Recorder) func(o *OS) { +// WithRecording configures OS to use the provided recording. +func WithRecording(r *recording.Recording) func(o *OS) { return func(o *OS) { - o.Recorder = r - } -} - -// WithWorkingDir configures Exec to use the provided working directory. -func WithWorkingDir(dir string) func(o *OS) { - return func(o *OS) { - o.dir = dir + o.Recording = r } } @@ -88,20 +71,14 @@ func (o *OS) MkdirAll(path string) error { command := fmt.Sprintf("mkdir %s", path) o.logger.Print(command) - if o.Recorder == nil || o.Recorder.Recording() { + if o.Recording == nil { // Do the real thing. if err := os.MkdirAll(path, 0755); err != nil { return err } - } - - if o.Recorder == nil { return nil } - if o.Recording() { - return o.record(command, "") - } _, err := o.replay(command) return err } @@ -111,20 +88,14 @@ func (o *OS) Remove(path string) error { command := fmt.Sprintf("rm %s", path) o.logger.Print(command) - if o.Recorder == nil || o.Recorder.Recording() { + if o.Recording == nil { // Do the real thing. if err := os.Remove(path); err != nil && !oserror.IsNotExist(err) { return err } - } - - if o.Recorder == nil { return nil } - if o.Recording() { - return o.record(command, "") - } _, err := o.replay(command) return err } @@ -135,20 +106,14 @@ func (o *OS) Symlink(to, from string) error { command := fmt.Sprintf("ln -s %s %s", to, from) o.logger.Print(command) - if o.Recorder == nil || o.Recorder.Recording() { + if o.Recording == nil { // Do the real thing. if err := os.Symlink(to, from); err != nil { return err } - } - - if o.Recorder == nil { return nil } - if o.Recording() { - return o.record(command, "") - } _, err := o.replay(command) return err } @@ -159,22 +124,11 @@ func (o OS) Getenv(key string) string { command := fmt.Sprintf("getenv %s", key) o.logger.Print(command) - var env string - if o.Recorder == nil || o.Recorder.Recording() { - env = os.Getenv(key) - } - - if o.Recorder == nil { - return env + if o.Recording == nil { + // Do the real thing. + return os.Getenv(key) } - if o.Recording() { - err := o.record(command, env) - if err != nil { - return "" - } - return env - } ret, _ := o.replay(command) return ret } @@ -185,20 +139,11 @@ func (o *OS) Setenv(key, value string) error { command := fmt.Sprintf("export %s=%s", key, value) o.logger.Print(command) - if o.Recorder == nil || o.Recorder.Recording() { + if o.Recording == nil { // Do the real thing. - if err := os.Setenv(key, value); err != nil { - return err - } - } - - if o.Recorder == nil { - return nil + return os.Setenv(key, value) } - if o.Recording() { - return o.record(command, "") - } _, err := o.replay(command) return err } @@ -209,23 +154,11 @@ func (o *OS) Readlink(filename string) (string, error) { command := fmt.Sprintf("readlink %s", filename) o.logger.Print(command) - var resolved string - if o.Recorder == nil || o.Recorder.Recording() { + if o.Recording == nil { // Do the real thing. - var err error - resolved, err = os.Readlink(filename) - if err != nil { - return "", err - } + return os.Readlink(filename) } - if o.Recorder == nil { - return resolved, nil - } - - if o.Recording() { - return resolved, o.record(command, resolved) - } ret, err := o.replay(command) return ret, err } @@ -236,31 +169,23 @@ func (o *OS) ReadFile(filename string) (string, error) { command := fmt.Sprintf("cat %s", filename) o.logger.Print(command) - var out string - if o.Recorder == nil || o.Recorder.Recording() { + if o.Recording == nil { // Do the real thing. buf, err := ioutil.ReadFile(filename) if err != nil { return "", err } - out = string(buf) - } - - if o.Recorder == nil { - return out, nil + return string(buf), nil } - if o.Recording() { - return out, o.record(command, out) - } ret, err := o.replay(command) return ret, err } // replay replays the specified command, erroring out if it's mismatched with -// what the recorder plays back next. It returns the recorded output. +// what the recording plays back next. It returns the recorded output. func (o *OS) replay(command string) (output string, err error) { - found, err := o.Recorder.Next(func(op recorder.Operation) error { + found, err := o.Recording.Next(func(op recording.Operation) error { if op.Command != command { return fmt.Errorf("expected %q, got %q", op.Command, command) } @@ -275,13 +200,3 @@ func (o *OS) replay(command string) (output string, err error) { } return output, nil } - -// record records the specified command. -func (o *OS) record(command, output string) error { - op := recorder.Operation{ - Command: command, - Output: output, - } - - return o.Record(op) -} diff --git a/pkg/cmd/dev/recorder/BUILD.bazel b/pkg/cmd/dev/recording/BUILD.bazel similarity index 80% rename from pkg/cmd/dev/recorder/BUILD.bazel rename to pkg/cmd/dev/recording/BUILD.bazel index 529ee3d45e99..19a47a095bb1 100644 --- a/pkg/cmd/dev/recorder/BUILD.bazel +++ b/pkg/cmd/dev/recording/BUILD.bazel @@ -1,12 +1,12 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( - name = "recorder", + name = "recording", srcs = [ "operation.go", - "recorder.go", + "recording.go", "scanner.go", ], - importpath = "github.com/cockroachdb/cockroach/pkg/cmd/dev/recorder", + importpath = "github.com/cockroachdb/cockroach/pkg/cmd/dev/recording", visibility = ["//visibility:public"], ) diff --git a/pkg/cmd/dev/recorder/operation.go b/pkg/cmd/dev/recording/operation.go similarity index 98% rename from pkg/cmd/dev/recorder/operation.go rename to pkg/cmd/dev/recording/operation.go index ff9d1f32b457..a14d775050e3 100644 --- a/pkg/cmd/dev/recorder/operation.go +++ b/pkg/cmd/dev/recording/operation.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package recorder +package recording import "strings" diff --git a/pkg/cmd/dev/recorder/recorder.go b/pkg/cmd/dev/recording/recording.go similarity index 69% rename from pkg/cmd/dev/recorder/recorder.go rename to pkg/cmd/dev/recording/recording.go index 6b81bb32421e..3cc87368ae46 100644 --- a/pkg/cmd/dev/recorder/recorder.go +++ b/pkg/cmd/dev/recording/recording.go @@ -8,79 +8,37 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package recorder +package recording import ( "bytes" - "errors" "fmt" "io" "strings" ) -// Recorder can be used to record a set of operations (defined only by a -// "command" and an "expected output"). These recordings can then be played -// back, which provides a handy way to mock out the components being recorded. -type Recorder struct { - // writer is set if we're in recording mode, and is where operations are - // recorded. - writer io.Writer - - // scanner and op are set if we're in replay mode. It's where we're - // replaying the recording from. op is the scratch space used to - // parse out the current operation being read. +// Recording can be used to play back a set of operations (defined only by a +// "command" and an "expected output"). It provides a handy way to mock out the +// components being recorded. +type Recording struct { + // scanner is where we're replaying the recording from. op is the + // scratch space used to parse out the current operation being read. scanner *scanner op Operation } -// New constructs a Recorder, using the specified configuration option (one of -// WithReplayFrom or WithRecordingTo). -func New(opt func(r *Recorder)) *Recorder { - r := &Recorder{} - opt(r) - return r -} - -// WithReplayFrom is used to configure a Recorder to play back from the given +// WithReplayFrom is used to configure a Recording to play back from the given // reader. The provided name is used only for diagnostic purposes, it's // typically the name of the file being read. -func WithReplayFrom(r io.Reader, name string) func(*Recorder) { - return func(re *Recorder) { - re.scanner = newScanner(r, name) - } -} - -// WithRecordingTo is used to configure a Recorder to record into the given -// writer. -func WithRecordingTo(w io.Writer) func(*Recorder) { - return func(r *Recorder) { - r.writer = w - } -} - -// Recording returns whether or not the recorder is configured to record (as -// opposed to replay from a recording). -func (r *Recorder) Recording() bool { - return r.writer != nil -} - -// Record is used to record the given operation. -func (r *Recorder) Record(o Operation) error { - if !r.Recording() { - return errors.New("misconfigured recorder; not set to record") - } - - _, err := r.writer.Write([]byte(o.String())) - return err +func WithReplayFrom(r io.Reader, name string) *Recording { + re := &Recording{} + re.scanner = newScanner(r, name) + return re } // Next is used to step through the next operation found in the recording, if // any. -func (r *Recorder) Next(f func(Operation) error) (found bool, err error) { - if r.Recording() { - return false, errors.New("misconfigured recorder; set to record, not replay") - } - +func (r *Recording) Next(f func(Operation) error) (found bool, err error) { parsed, err := r.parseOperation() if err != nil { return false, err @@ -99,7 +57,7 @@ func (r *Recorder) Next(f func(Operation) error) (found bool, err error) { // parseOperation parses out the next Operation from the internal scanner. See // type-level comment on Operation to understand the grammar we're parsing // against. -func (r *Recorder) parseOperation() (parsed bool, err error) { +func (r *Recording) parseOperation() (parsed bool, err error) { for r.scanner.Scan() { r.op = Operation{} line := r.scanner.Text() @@ -144,7 +102,7 @@ func (r *Recorder) parseOperation() (parsed bool, err error) { // parseCommand parses a line and returns it if parsed correctly. See // type-level comment on Operation to understand the grammar we're parsing // against. -func (r *Recorder) parseCommand(line string) (cmd string, err error) { +func (r *Recording) parseCommand(line string) (cmd string, err error) { line = strings.TrimSpace(line) if line == "" { return "", nil @@ -162,7 +120,7 @@ func (r *Recorder) parseCommand(line string) (cmd string, err error) { // parseSeparator parses a separator ('----'), erroring out if it's not parsed // correctly. See type-level comment on Operation to understand the grammar // we're parsing against. -func (r *Recorder) parseSeparator() error { +func (r *Recording) parseSeparator() error { if !r.scanner.Scan() { return fmt.Errorf("%s: expected to find separator after command", r.scanner.pos()) } @@ -175,7 +133,7 @@ func (r *Recorder) parseSeparator() error { // parseOutput parses an . See type-level comment on Operation to // understand the grammar we're parsing against. -func (r *Recorder) parseOutput() error { +func (r *Recording) parseOutput() error { var buf bytes.Buffer var line string diff --git a/pkg/cmd/dev/recorder/scanner.go b/pkg/cmd/dev/recording/scanner.go similarity index 98% rename from pkg/cmd/dev/recorder/scanner.go rename to pkg/cmd/dev/recording/scanner.go index d0d5a322c9d7..06d2c8c22764 100644 --- a/pkg/cmd/dev/recorder/scanner.go +++ b/pkg/cmd/dev/recording/scanner.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package recorder +package recording import ( "bufio" diff --git a/pkg/cmd/dev/test.go b/pkg/cmd/dev/test.go index aaa316936e18..605a7a5b7c6e 100644 --- a/pkg/cmd/dev/test.go +++ b/pkg/cmd/dev/test.go @@ -186,7 +186,7 @@ func (d *dev) runUnitTest(cmd *cobra.Command, pkgs []string) error { args = append(args, "--test_output", "errors") } - err := d.exec.CommandContextNoRecord(ctx, "bazel", args...) + err := d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...) return err } From 9f0747f37f2b70eb09553008cabc2a373bb2b73b Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Fri, 30 Jul 2021 18:50:22 +0200 Subject: [PATCH 8/8] cli/zip: avoid quadratic behavior in SQL schema retrieval Prior to this patch, the `debug zip` command would issue a `DatabaseDetails` request over the network for each database in turn, and also `TableDetails` for each table. Server-side, each of these individual requests would cause all descriptors to be loaded into memory first. The overall behavior was thus quadratic in performance, and was resulting in unacceptable delays on large clusters with thousands of descriptors. It turns out, none of this is necessary since we already collect the descriptor details via `system.descriptors` and can collect the CREATE statements trivially using the `crdb_internal.create_statements` vtable, which also has linear performance. Release note (cli change): `cockroach debug zip` no longer retrieves database and table details into separate files. The schema information is collected by means of `system.descriptors` and `crdb_internal.create_statements`. --- pkg/cli/testdata/zip/partial1 | 46 +----- pkg/cli/testdata/zip/partial1_excluded | 46 +----- pkg/cli/testdata/zip/partial2 | 46 +----- pkg/cli/testdata/zip/specialnames | 58 ------- pkg/cli/testdata/zip/testzip | 48 +----- pkg/cli/testdata/zip/testzip_concurrent | 206 +----------------------- pkg/cli/testdata/zip/unavailable | 42 +++-- pkg/cli/zip.go | 11 +- pkg/cli/zip_cluster_wide.go | 83 +--------- pkg/cli/zip_helpers.go | 37 ----- pkg/cli/zip_test.go | 17 +- 11 files changed, 64 insertions(+), 576 deletions(-) diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index fa08ac6b10ec..b121ebfd484c 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -22,6 +22,8 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] retrieving SQL data for system.descriptor... writing output: debug/system.descriptor.txt... done [cluster] retrieving SQL data for system.namespace... writing output: debug/system.namespace.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done @@ -272,49 +274,5 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done [node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done [node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done -[cluster] doctor examining cluster...... writing binary output: debug/reports/doctor.txt... done -[cluster] requesting list of SQL databases... received response... done -[cluster] 3 databases found -[cluster] [database: defaultdb] requesting database details... received response... converting to JSON... writing binary output: debug/schema/defaultdb@details.json... done -[cluster] [database: defaultdb] 0 tables found -[cluster] [database: postgres] requesting database details... received response... converting to JSON... writing binary output: debug/schema/postgres@details.json... done -[cluster] [database: postgres] 0 tables found -[cluster] [database: system] requesting database details... received response... converting to JSON... writing binary output: debug/schema/system@details.json... done -[cluster] [database: system] 35 tables found -[cluster] [database: system] [table: public.comments] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_comments.json... done -[cluster] [database: system] [table: public.database_role_settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_database_role_settings.json... done -[cluster] [database: system] [table: public.descriptor] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_descriptor.json... done -[cluster] [database: system] [table: public.eventlog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_eventlog.json... done -[cluster] [database: system] [table: public.jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_jobs.json... done -[cluster] [database: system] [table: public.join_tokens] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_join_tokens.json... done -[cluster] [database: system] [table: public.lease] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_lease.json... done -[cluster] [database: system] [table: public.locations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_locations.json... done -[cluster] [database: system] [table: public.migrations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_migrations.json... done -[cluster] [database: system] [table: public.namespace] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_namespace.json... done -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_meta.json... done -[cluster] [database: system] [table: public.protected_ts_records] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_records.json... done -[cluster] [database: system] [table: public.rangelog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_rangelog.json... done -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_constraint_stats.json... done -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_critical_localities.json... done -[cluster] [database: system] [table: public.replication_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_stats.json... done -[cluster] [database: system] [table: public.reports_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_reports_meta.json... done -[cluster] [database: system] [table: public.role_members] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_members.json... done -[cluster] [database: system] [table: public.role_options] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_options.json... done -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_scheduled_jobs.json... done -[cluster] [database: system] [table: public.settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_settings.json... done -[cluster] [database: system] [table: public.sql_instances] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sql_instances.json... done -[cluster] [database: system] [table: public.sqlliveness] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sqlliveness.json... done -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_bundle_chunks.json... done -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics.json... done -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics_requests.json... done -[cluster] [database: system] [table: public.statement_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_statistics.json... done -[cluster] [database: system] [table: public.table_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_table_statistics.json... done -[cluster] [database: system] [table: public.tenant_usage] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenant_usage.json... done -[cluster] [database: system] [table: public.tenants] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenants.json... done -[cluster] [database: system] [table: public.transaction_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_transaction_statistics.json... done -[cluster] [database: system] [table: public.ui] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_ui.json... done -[cluster] [database: system] [table: public.users] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_users.json... done -[cluster] [database: system] [table: public.web_sessions] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_web_sessions.json... done -[cluster] [database: system] [table: public.zones] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_zones.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index a9a165db18a4..dbe7d896619e 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -22,6 +22,8 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] retrieving SQL data for system.descriptor... writing output: debug/system.descriptor.txt... done [cluster] retrieving SQL data for system.namespace... writing output: debug/system.namespace.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done @@ -190,49 +192,5 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done [node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done [node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done -[cluster] doctor examining cluster...... writing binary output: debug/reports/doctor.txt... done -[cluster] requesting list of SQL databases... received response... done -[cluster] 3 databases found -[cluster] [database: defaultdb] requesting database details... received response... converting to JSON... writing binary output: debug/schema/defaultdb@details.json... done -[cluster] [database: defaultdb] 0 tables found -[cluster] [database: postgres] requesting database details... received response... converting to JSON... writing binary output: debug/schema/postgres@details.json... done -[cluster] [database: postgres] 0 tables found -[cluster] [database: system] requesting database details... received response... converting to JSON... writing binary output: debug/schema/system@details.json... done -[cluster] [database: system] 35 tables found -[cluster] [database: system] [table: public.comments] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_comments.json... done -[cluster] [database: system] [table: public.database_role_settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_database_role_settings.json... done -[cluster] [database: system] [table: public.descriptor] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_descriptor.json... done -[cluster] [database: system] [table: public.eventlog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_eventlog.json... done -[cluster] [database: system] [table: public.jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_jobs.json... done -[cluster] [database: system] [table: public.join_tokens] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_join_tokens.json... done -[cluster] [database: system] [table: public.lease] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_lease.json... done -[cluster] [database: system] [table: public.locations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_locations.json... done -[cluster] [database: system] [table: public.migrations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_migrations.json... done -[cluster] [database: system] [table: public.namespace] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_namespace.json... done -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_meta.json... done -[cluster] [database: system] [table: public.protected_ts_records] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_records.json... done -[cluster] [database: system] [table: public.rangelog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_rangelog.json... done -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_constraint_stats.json... done -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_critical_localities.json... done -[cluster] [database: system] [table: public.replication_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_stats.json... done -[cluster] [database: system] [table: public.reports_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_reports_meta.json... done -[cluster] [database: system] [table: public.role_members] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_members.json... done -[cluster] [database: system] [table: public.role_options] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_options.json... done -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_scheduled_jobs.json... done -[cluster] [database: system] [table: public.settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_settings.json... done -[cluster] [database: system] [table: public.sql_instances] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sql_instances.json... done -[cluster] [database: system] [table: public.sqlliveness] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sqlliveness.json... done -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_bundle_chunks.json... done -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics.json... done -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics_requests.json... done -[cluster] [database: system] [table: public.statement_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_statistics.json... done -[cluster] [database: system] [table: public.table_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_table_statistics.json... done -[cluster] [database: system] [table: public.tenant_usage] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenant_usage.json... done -[cluster] [database: system] [table: public.tenants] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenants.json... done -[cluster] [database: system] [table: public.transaction_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_transaction_statistics.json... done -[cluster] [database: system] [table: public.ui] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_ui.json... done -[cluster] [database: system] [table: public.users] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_users.json... done -[cluster] [database: system] [table: public.web_sessions] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_web_sessions.json... done -[cluster] [database: system] [table: public.zones] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_zones.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index a2a9147ff0b6..c0807332419d 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -22,6 +22,8 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] retrieving SQL data for system.descriptor... writing output: debug/system.descriptor.txt... done [cluster] retrieving SQL data for system.namespace... writing output: debug/system.namespace.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done @@ -189,49 +191,5 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [node 3] writing range 40... converting to JSON... writing binary output: debug/nodes/3/ranges/40.json... done [node 3] writing range 41... converting to JSON... writing binary output: debug/nodes/3/ranges/41.json... done [node 3] writing range 42... converting to JSON... writing binary output: debug/nodes/3/ranges/42.json... done -[cluster] doctor examining cluster...... writing binary output: debug/reports/doctor.txt... done -[cluster] requesting list of SQL databases... received response... done -[cluster] 3 databases found -[cluster] [database: defaultdb] requesting database details... received response... converting to JSON... writing binary output: debug/schema/defaultdb@details.json... done -[cluster] [database: defaultdb] 0 tables found -[cluster] [database: postgres] requesting database details... received response... converting to JSON... writing binary output: debug/schema/postgres@details.json... done -[cluster] [database: postgres] 0 tables found -[cluster] [database: system] requesting database details... received response... converting to JSON... writing binary output: debug/schema/system@details.json... done -[cluster] [database: system] 35 tables found -[cluster] [database: system] [table: public.comments] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_comments.json... done -[cluster] [database: system] [table: public.database_role_settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_database_role_settings.json... done -[cluster] [database: system] [table: public.descriptor] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_descriptor.json... done -[cluster] [database: system] [table: public.eventlog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_eventlog.json... done -[cluster] [database: system] [table: public.jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_jobs.json... done -[cluster] [database: system] [table: public.join_tokens] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_join_tokens.json... done -[cluster] [database: system] [table: public.lease] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_lease.json... done -[cluster] [database: system] [table: public.locations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_locations.json... done -[cluster] [database: system] [table: public.migrations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_migrations.json... done -[cluster] [database: system] [table: public.namespace] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_namespace.json... done -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_meta.json... done -[cluster] [database: system] [table: public.protected_ts_records] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_records.json... done -[cluster] [database: system] [table: public.rangelog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_rangelog.json... done -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_constraint_stats.json... done -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_critical_localities.json... done -[cluster] [database: system] [table: public.replication_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_stats.json... done -[cluster] [database: system] [table: public.reports_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_reports_meta.json... done -[cluster] [database: system] [table: public.role_members] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_members.json... done -[cluster] [database: system] [table: public.role_options] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_options.json... done -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_scheduled_jobs.json... done -[cluster] [database: system] [table: public.settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_settings.json... done -[cluster] [database: system] [table: public.sql_instances] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sql_instances.json... done -[cluster] [database: system] [table: public.sqlliveness] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sqlliveness.json... done -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_bundle_chunks.json... done -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics.json... done -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics_requests.json... done -[cluster] [database: system] [table: public.statement_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_statistics.json... done -[cluster] [database: system] [table: public.table_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_table_statistics.json... done -[cluster] [database: system] [table: public.tenant_usage] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenant_usage.json... done -[cluster] [database: system] [table: public.tenants] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenants.json... done -[cluster] [database: system] [table: public.transaction_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_transaction_statistics.json... done -[cluster] [database: system] [table: public.ui] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_ui.json... done -[cluster] [database: system] [table: public.users] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_users.json... done -[cluster] [database: system] [table: public.web_sessions] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_web_sessions.json... done -[cluster] [database: system] [table: public.zones] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_zones.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/specialnames b/pkg/cli/testdata/zip/specialnames index 95c57b01ff68..27b6a4c1b569 100644 --- a/pkg/cli/testdata/zip/specialnames +++ b/pkg/cli/testdata/zip/specialnames @@ -1,61 +1,3 @@ zip ---- [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done -[cluster] requesting list of SQL databases... received response... done -[cluster] 8 databases found -[cluster] [database: ../system] requesting database details... received response... converting to JSON... writing binary output: debug/schema/___system@details.json... done -[cluster] [database: ../system] 0 tables found -[cluster] [database: SYSTEM] requesting database details... received response... converting to JSON... writing binary output: debug/schema/system@details.json... done -[cluster] [database: SYSTEM] 0 tables found -[cluster] [database: a-b] requesting database details... received response... converting to JSON... writing binary output: debug/schema/a_b@details.json... done -[cluster] [database: a-b] 0 tables found -[cluster] [database: a-b-1] requesting database details... received response... converting to JSON... writing binary output: debug/schema/a_b_1@details.json... done -[cluster] [database: a-b-1] 0 tables found -[cluster] [database: a:b] requesting database details... received response... converting to JSON... writing binary output: debug/schema/a_b-1@details.json... done -[cluster] [database: a:b] 0 tables found -[cluster] [database: defaultdb] requesting database details... received response... converting to JSON... writing binary output: debug/schema/defaultdb@details.json... done -[cluster] [database: defaultdb] 5 tables found -[cluster] [database: defaultdb] [table: public."../system"] requesting table details... received response... converting to JSON... writing binary output: debug/schema/defaultdb/public_____system_.json... done -[cluster] [database: defaultdb] [table: public."SYSTEM.JOBS"] requesting table details... received response... converting to JSON... writing binary output: debug/schema/defaultdb/public__system_jobs_.json... done -[cluster] [database: defaultdb] [table: public."a-b"] requesting table details... received response... converting to JSON... writing binary output: debug/schema/defaultdb/public__a_b_.json... done -[cluster] [database: defaultdb] [table: public."a:b"] requesting table details... received response... converting to JSON... writing binary output: debug/schema/defaultdb/public__a_b_-1.json... done -[cluster] [database: defaultdb] [table: public."pg_catalog.pg_class"] requesting table details... received response... converting to JSON... writing binary output: debug/schema/defaultdb/public__pg_catalog_pg_class_.json... done -[cluster] [database: postgres] requesting database details... received response... converting to JSON... writing binary output: debug/schema/postgres@details.json... done -[cluster] [database: postgres] 0 tables found -[cluster] [database: system] requesting database details... received response... converting to JSON... writing binary output: debug/schema/system-1@details.json... done -[cluster] [database: system] 35 tables found -[cluster] [database: system] [table: public.comments] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_comments.json... done -[cluster] [database: system] [table: public.database_role_settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_database_role_settings.json... done -[cluster] [database: system] [table: public.descriptor] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_descriptor.json... done -[cluster] [database: system] [table: public.eventlog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_eventlog.json... done -[cluster] [database: system] [table: public.jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_jobs.json... done -[cluster] [database: system] [table: public.join_tokens] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_join_tokens.json... done -[cluster] [database: system] [table: public.lease] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_lease.json... done -[cluster] [database: system] [table: public.locations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_locations.json... done -[cluster] [database: system] [table: public.migrations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_migrations.json... done -[cluster] [database: system] [table: public.namespace] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_namespace.json... done -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_protected_ts_meta.json... done -[cluster] [database: system] [table: public.protected_ts_records] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_protected_ts_records.json... done -[cluster] [database: system] [table: public.rangelog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_rangelog.json... done -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_replication_constraint_stats.json... done -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_replication_critical_localities.json... done -[cluster] [database: system] [table: public.replication_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_replication_stats.json... done -[cluster] [database: system] [table: public.reports_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_reports_meta.json... done -[cluster] [database: system] [table: public.role_members] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_role_members.json... done -[cluster] [database: system] [table: public.role_options] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_role_options.json... done -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_scheduled_jobs.json... done -[cluster] [database: system] [table: public.settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_settings.json... done -[cluster] [database: system] [table: public.sql_instances] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_sql_instances.json... done -[cluster] [database: system] [table: public.sqlliveness] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_sqlliveness.json... done -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_statement_bundle_chunks.json... done -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_statement_diagnostics.json... done -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_statement_diagnostics_requests.json... done -[cluster] [database: system] [table: public.statement_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_statement_statistics.json... done -[cluster] [database: system] [table: public.table_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_table_statistics.json... done -[cluster] [database: system] [table: public.tenant_usage] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_tenant_usage.json... done -[cluster] [database: system] [table: public.tenants] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_tenants.json... done -[cluster] [database: system] [table: public.transaction_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_transaction_statistics.json... done -[cluster] [database: system] [table: public.ui] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_ui.json... done -[cluster] [database: system] [table: public.users] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_users.json... done -[cluster] [database: system] [table: public.web_sessions] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_web_sessions.json... done -[cluster] [database: system] [table: public.zones] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system-1/public_zones.json... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index a8c3d6df4af1..d18653d7c5e0 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -22,6 +22,8 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for system.descriptor... writing output: debug/system.descriptor.txt... done [cluster] retrieving SQL data for system.namespace... writing output: debug/system.namespace.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... done @@ -62,7 +64,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] requesting stacks... received response... writing binary output: debug/nodes/1/stacks.txt... done [node 1] requesting heap profile... received response... writing binary output: debug/nodes/1/heap.pprof... done [node 1] requesting heap file list... received response... done -[node 1] 0 heap profiles found +[node ?] ? heap profiles found [node 1] requesting goroutine dump list... received response... done [node 1] 0 goroutine dumps found [node 1] requesting log file ... @@ -111,49 +113,5 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done [node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done [node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done -[cluster] doctor examining cluster...... writing binary output: debug/reports/doctor.txt... done -[cluster] requesting list of SQL databases... received response... done -[cluster] 3 databases found -[cluster] [database: defaultdb] requesting database details... received response... converting to JSON... writing binary output: debug/schema/defaultdb@details.json... done -[cluster] [database: defaultdb] 0 tables found -[cluster] [database: postgres] requesting database details... received response... converting to JSON... writing binary output: debug/schema/postgres@details.json... done -[cluster] [database: postgres] 0 tables found -[cluster] [database: system] requesting database details... received response... converting to JSON... writing binary output: debug/schema/system@details.json... done -[cluster] [database: system] 35 tables found -[cluster] [database: system] [table: public.comments] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_comments.json... done -[cluster] [database: system] [table: public.database_role_settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_database_role_settings.json... done -[cluster] [database: system] [table: public.descriptor] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_descriptor.json... done -[cluster] [database: system] [table: public.eventlog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_eventlog.json... done -[cluster] [database: system] [table: public.jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_jobs.json... done -[cluster] [database: system] [table: public.join_tokens] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_join_tokens.json... done -[cluster] [database: system] [table: public.lease] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_lease.json... done -[cluster] [database: system] [table: public.locations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_locations.json... done -[cluster] [database: system] [table: public.migrations] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_migrations.json... done -[cluster] [database: system] [table: public.namespace] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_namespace.json... done -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_meta.json... done -[cluster] [database: system] [table: public.protected_ts_records] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_protected_ts_records.json... done -[cluster] [database: system] [table: public.rangelog] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_rangelog.json... done -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_constraint_stats.json... done -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_critical_localities.json... done -[cluster] [database: system] [table: public.replication_stats] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_replication_stats.json... done -[cluster] [database: system] [table: public.reports_meta] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_reports_meta.json... done -[cluster] [database: system] [table: public.role_members] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_members.json... done -[cluster] [database: system] [table: public.role_options] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_role_options.json... done -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_scheduled_jobs.json... done -[cluster] [database: system] [table: public.settings] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_settings.json... done -[cluster] [database: system] [table: public.sql_instances] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sql_instances.json... done -[cluster] [database: system] [table: public.sqlliveness] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_sqlliveness.json... done -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_bundle_chunks.json... done -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics.json... done -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_diagnostics_requests.json... done -[cluster] [database: system] [table: public.statement_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_statement_statistics.json... done -[cluster] [database: system] [table: public.table_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_table_statistics.json... done -[cluster] [database: system] [table: public.tenant_usage] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenant_usage.json... done -[cluster] [database: system] [table: public.tenants] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_tenants.json... done -[cluster] [database: system] [table: public.transaction_statistics] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_transaction_statistics.json... done -[cluster] [database: system] [table: public.ui] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_ui.json... done -[cluster] [database: system] [table: public.users] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_users.json... done -[cluster] [database: system] [table: public.web_sessions] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_web_sessions.json... done -[cluster] [database: system] [table: public.zones] requesting table details... received response... converting to JSON... writing binary output: debug/schema/system/public_zones.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index a9b8363d82b7..7f71695b740f 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -1,204 +1,7 @@ zip ---- -[cluster] 3 databases found -[cluster] [database: defaultdb] 0 tables found -[cluster] [database: defaultdb] requesting database details... -[cluster] [database: defaultdb] requesting database details: converting to JSON... -[cluster] [database: defaultdb] requesting database details: done -[cluster] [database: defaultdb] requesting database details: received response... -[cluster] [database: defaultdb] requesting database details: writing binary output: debug/schema/defaultdb@details.json... -[cluster] [database: postgres] 0 tables found -[cluster] [database: postgres] requesting database details... -[cluster] [database: postgres] requesting database details: converting to JSON... -[cluster] [database: postgres] requesting database details: done -[cluster] [database: postgres] requesting database details: received response... -[cluster] [database: postgres] requesting database details: writing binary output: debug/schema/postgres@details.json... -[cluster] [database: system] 35 tables found -[cluster] [database: system] [table: public.comments] requesting table details... -[cluster] [database: system] [table: public.comments] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.comments] requesting table details: done -[cluster] [database: system] [table: public.comments] requesting table details: received response... -[cluster] [database: system] [table: public.comments] requesting table details: writing binary output: debug/schema/system/public_comments.json... -[cluster] [database: system] [table: public.database_role_settings] requesting table details... -[cluster] [database: system] [table: public.database_role_settings] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.database_role_settings] requesting table details: done -[cluster] [database: system] [table: public.database_role_settings] requesting table details: received response... -[cluster] [database: system] [table: public.database_role_settings] requesting table details: writing binary output: debug/schema/system/public_database_role_settings.json... -[cluster] [database: system] [table: public.descriptor] requesting table details... -[cluster] [database: system] [table: public.descriptor] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.descriptor] requesting table details: done -[cluster] [database: system] [table: public.descriptor] requesting table details: received response... -[cluster] [database: system] [table: public.descriptor] requesting table details: writing binary output: debug/schema/system/public_descriptor.json... -[cluster] [database: system] [table: public.eventlog] requesting table details... -[cluster] [database: system] [table: public.eventlog] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.eventlog] requesting table details: done -[cluster] [database: system] [table: public.eventlog] requesting table details: received response... -[cluster] [database: system] [table: public.eventlog] requesting table details: writing binary output: debug/schema/system/public_eventlog.json... -[cluster] [database: system] [table: public.jobs] requesting table details... -[cluster] [database: system] [table: public.jobs] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.jobs] requesting table details: done -[cluster] [database: system] [table: public.jobs] requesting table details: received response... -[cluster] [database: system] [table: public.jobs] requesting table details: writing binary output: debug/schema/system/public_jobs.json... -[cluster] [database: system] [table: public.join_tokens] requesting table details... -[cluster] [database: system] [table: public.join_tokens] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.join_tokens] requesting table details: done -[cluster] [database: system] [table: public.join_tokens] requesting table details: received response... -[cluster] [database: system] [table: public.join_tokens] requesting table details: writing binary output: debug/schema/system/public_join_tokens.json... -[cluster] [database: system] [table: public.lease] requesting table details... -[cluster] [database: system] [table: public.lease] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.lease] requesting table details: done -[cluster] [database: system] [table: public.lease] requesting table details: received response... -[cluster] [database: system] [table: public.lease] requesting table details: writing binary output: debug/schema/system/public_lease.json... -[cluster] [database: system] [table: public.locations] requesting table details... -[cluster] [database: system] [table: public.locations] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.locations] requesting table details: done -[cluster] [database: system] [table: public.locations] requesting table details: received response... -[cluster] [database: system] [table: public.locations] requesting table details: writing binary output: debug/schema/system/public_locations.json... -[cluster] [database: system] [table: public.migrations] requesting table details... -[cluster] [database: system] [table: public.migrations] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.migrations] requesting table details: done -[cluster] [database: system] [table: public.migrations] requesting table details: received response... -[cluster] [database: system] [table: public.migrations] requesting table details: writing binary output: debug/schema/system/public_migrations.json... -[cluster] [database: system] [table: public.namespace] requesting table details... -[cluster] [database: system] [table: public.namespace] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.namespace] requesting table details: done -[cluster] [database: system] [table: public.namespace] requesting table details: received response... -[cluster] [database: system] [table: public.namespace] requesting table details: writing binary output: debug/schema/system/public_namespace.json... -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details... -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details: done -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details: received response... -[cluster] [database: system] [table: public.protected_ts_meta] requesting table details: writing binary output: debug/schema/system/public_protected_ts_meta.json... -[cluster] [database: system] [table: public.protected_ts_records] requesting table details... -[cluster] [database: system] [table: public.protected_ts_records] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.protected_ts_records] requesting table details: done -[cluster] [database: system] [table: public.protected_ts_records] requesting table details: received response... -[cluster] [database: system] [table: public.protected_ts_records] requesting table details: writing binary output: debug/schema/system/public_protected_ts_records.json... -[cluster] [database: system] [table: public.rangelog] requesting table details... -[cluster] [database: system] [table: public.rangelog] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.rangelog] requesting table details: done -[cluster] [database: system] [table: public.rangelog] requesting table details: received response... -[cluster] [database: system] [table: public.rangelog] requesting table details: writing binary output: debug/schema/system/public_rangelog.json... -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details... -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details: done -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details: received response... -[cluster] [database: system] [table: public.replication_constraint_stats] requesting table details: writing binary output: debug/schema/system/public_replication_constraint_stats.json... -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details... -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details: done -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details: received response... -[cluster] [database: system] [table: public.replication_critical_localities] requesting table details: writing binary output: debug/schema/system/public_replication_critical_localities.json... -[cluster] [database: system] [table: public.replication_stats] requesting table details... -[cluster] [database: system] [table: public.replication_stats] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.replication_stats] requesting table details: done -[cluster] [database: system] [table: public.replication_stats] requesting table details: received response... -[cluster] [database: system] [table: public.replication_stats] requesting table details: writing binary output: debug/schema/system/public_replication_stats.json... -[cluster] [database: system] [table: public.reports_meta] requesting table details... -[cluster] [database: system] [table: public.reports_meta] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.reports_meta] requesting table details: done -[cluster] [database: system] [table: public.reports_meta] requesting table details: received response... -[cluster] [database: system] [table: public.reports_meta] requesting table details: writing binary output: debug/schema/system/public_reports_meta.json... -[cluster] [database: system] [table: public.role_members] requesting table details... -[cluster] [database: system] [table: public.role_members] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.role_members] requesting table details: done -[cluster] [database: system] [table: public.role_members] requesting table details: received response... -[cluster] [database: system] [table: public.role_members] requesting table details: writing binary output: debug/schema/system/public_role_members.json... -[cluster] [database: system] [table: public.role_options] requesting table details... -[cluster] [database: system] [table: public.role_options] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.role_options] requesting table details: done -[cluster] [database: system] [table: public.role_options] requesting table details: received response... -[cluster] [database: system] [table: public.role_options] requesting table details: writing binary output: debug/schema/system/public_role_options.json... -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details... -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details: done -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details: received response... -[cluster] [database: system] [table: public.scheduled_jobs] requesting table details: writing binary output: debug/schema/system/public_scheduled_jobs.json... -[cluster] [database: system] [table: public.settings] requesting table details... -[cluster] [database: system] [table: public.settings] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.settings] requesting table details: done -[cluster] [database: system] [table: public.settings] requesting table details: received response... -[cluster] [database: system] [table: public.settings] requesting table details: writing binary output: debug/schema/system/public_settings.json... -[cluster] [database: system] [table: public.sql_instances] requesting table details... -[cluster] [database: system] [table: public.sql_instances] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.sql_instances] requesting table details: done -[cluster] [database: system] [table: public.sql_instances] requesting table details: received response... -[cluster] [database: system] [table: public.sql_instances] requesting table details: writing binary output: debug/schema/system/public_sql_instances.json... -[cluster] [database: system] [table: public.sqlliveness] requesting table details... -[cluster] [database: system] [table: public.sqlliveness] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.sqlliveness] requesting table details: done -[cluster] [database: system] [table: public.sqlliveness] requesting table details: received response... -[cluster] [database: system] [table: public.sqlliveness] requesting table details: writing binary output: debug/schema/system/public_sqlliveness.json... -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details... -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details: done -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details: received response... -[cluster] [database: system] [table: public.statement_bundle_chunks] requesting table details: writing binary output: debug/schema/system/public_statement_bundle_chunks.json... -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details... -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details: done -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details: received response... -[cluster] [database: system] [table: public.statement_diagnostics] requesting table details: writing binary output: debug/schema/system/public_statement_diagnostics.json... -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details... -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details: done -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details: received response... -[cluster] [database: system] [table: public.statement_diagnostics_requests] requesting table details: writing binary output: debug/schema/system/public_statement_diagnostics_requests.json... -[cluster] [database: system] [table: public.statement_statistics] requesting table details... -[cluster] [database: system] [table: public.statement_statistics] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.statement_statistics] requesting table details: done -[cluster] [database: system] [table: public.statement_statistics] requesting table details: received response... -[cluster] [database: system] [table: public.statement_statistics] requesting table details: writing binary output: debug/schema/system/public_statement_statistics.json... -[cluster] [database: system] [table: public.table_statistics] requesting table details... -[cluster] [database: system] [table: public.table_statistics] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.table_statistics] requesting table details: done -[cluster] [database: system] [table: public.table_statistics] requesting table details: received response... -[cluster] [database: system] [table: public.table_statistics] requesting table details: writing binary output: debug/schema/system/public_table_statistics.json... -[cluster] [database: system] [table: public.tenant_usage] requesting table details... -[cluster] [database: system] [table: public.tenant_usage] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.tenant_usage] requesting table details: done -[cluster] [database: system] [table: public.tenant_usage] requesting table details: received response... -[cluster] [database: system] [table: public.tenant_usage] requesting table details: writing binary output: debug/schema/system/public_tenant_usage.json... -[cluster] [database: system] [table: public.tenants] requesting table details... -[cluster] [database: system] [table: public.tenants] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.tenants] requesting table details: done -[cluster] [database: system] [table: public.tenants] requesting table details: received response... -[cluster] [database: system] [table: public.tenants] requesting table details: writing binary output: debug/schema/system/public_tenants.json... -[cluster] [database: system] [table: public.transaction_statistics] requesting table details... -[cluster] [database: system] [table: public.transaction_statistics] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.transaction_statistics] requesting table details: done -[cluster] [database: system] [table: public.transaction_statistics] requesting table details: received response... -[cluster] [database: system] [table: public.transaction_statistics] requesting table details: writing binary output: debug/schema/system/public_transaction_statistics.json... -[cluster] [database: system] [table: public.ui] requesting table details... -[cluster] [database: system] [table: public.ui] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.ui] requesting table details: done -[cluster] [database: system] [table: public.ui] requesting table details: received response... -[cluster] [database: system] [table: public.ui] requesting table details: writing binary output: debug/schema/system/public_ui.json... -[cluster] [database: system] [table: public.users] requesting table details... -[cluster] [database: system] [table: public.users] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.users] requesting table details: done -[cluster] [database: system] [table: public.users] requesting table details: received response... -[cluster] [database: system] [table: public.users] requesting table details: writing binary output: debug/schema/system/public_users.json... -[cluster] [database: system] [table: public.web_sessions] requesting table details... -[cluster] [database: system] [table: public.web_sessions] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.web_sessions] requesting table details: done -[cluster] [database: system] [table: public.web_sessions] requesting table details: received response... -[cluster] [database: system] [table: public.web_sessions] requesting table details: writing binary output: debug/schema/system/public_web_sessions.json... -[cluster] [database: system] [table: public.zones] requesting table details... -[cluster] [database: system] [table: public.zones] requesting table details: converting to JSON... -[cluster] [database: system] [table: public.zones] requesting table details: done -[cluster] [database: system] [table: public.zones] requesting table details: received response... -[cluster] [database: system] [table: public.zones] requesting table details: writing binary output: debug/schema/system/public_zones.json... -[cluster] [database: system] requesting database details... -[cluster] [database: system] requesting database details: converting to JSON... -[cluster] [database: system] requesting database details: done -[cluster] [database: system] requesting database details: received response... -[cluster] [database: system] requesting database details: writing binary output: debug/schema/system@details.json... [cluster] creating output file /dev/null... [cluster] creating output file /dev/null: done -[cluster] doctor examining cluster...... -[cluster] doctor examining cluster...: done -[cluster] doctor examining cluster...: writing binary output: debug/reports/doctor.txt... [cluster] establishing RPC connection to ... [cluster] establishing RPC connection to ... [cluster] hot range summary script... @@ -227,9 +30,6 @@ zip [cluster] requesting data for debug/settings: done [cluster] requesting data for debug/settings: received response... [cluster] requesting data for debug/settings: writing binary output: debug/settings.json... -[cluster] requesting list of SQL databases... -[cluster] requesting list of SQL databases: done -[cluster] requesting list of SQL databases: received response... [cluster] requesting liveness... [cluster] requesting liveness: converting to JSON... [cluster] requesting liveness: done @@ -240,6 +40,12 @@ zip [cluster] requesting nodes: done [cluster] requesting nodes: received response... [cluster] requesting nodes: writing binary output: debug/nodes.json... +[cluster] retrieving SQL data for "".crdb_internal.create_statements... +[cluster] retrieving SQL data for "".crdb_internal.create_statements: done +[cluster] retrieving SQL data for "".crdb_internal.create_statements: writing output: debug/crdb_internal.create_statements.txt... +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements... +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements: done +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements: writing output: debug/crdb_internal.create_type_statements.txt... [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... [cluster] retrieving SQL data for crdb_internal.cluster_contention_events: done [cluster] retrieving SQL data for crdb_internal.cluster_contention_events: writing output: debug/crdb_internal.cluster_contention_events.txt... diff --git a/pkg/cli/testdata/zip/unavailable b/pkg/cli/testdata/zip/unavailable index 0b6e86f5453a..f48e3c425dab 100644 --- a/pkg/cli/testdata/zip/unavailable +++ b/pkg/cli/testdata/zip/unavailable @@ -6,10 +6,10 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null --timeout=.5s [cluster] using SQL address: ... [cluster] creating output file /dev/null... done [cluster] requesting data for debug/events... received response... -[cluster] requesting data for debug/events: last request failed: operation "[cluster] requesting data for debug/events" timed out after 500ms +[cluster] requesting data for debug/events: last request failed: operation "[cluster] requesting data for debug/events" timed out after 500ms: rpc error: ... [cluster] requesting data for debug/events: creating error output: debug/events.json.err.txt... done [cluster] requesting data for debug/rangelog... received response... -[cluster] requesting data for debug/rangelog: last request failed: operation "[cluster] requesting data for debug/rangelog" timed out after 500ms +[cluster] requesting data for debug/rangelog: last request failed: operation "[cluster] requesting data for debug/rangelog" timed out after 500ms: rpc error: ... [cluster] requesting data for debug/rangelog: creating error output: debug/rangelog.json.err.txt... done [cluster] requesting data for debug/settings... received response... converting to JSON... writing binary output: debug/settings.json... done [cluster] requesting data for debug/reports/problemranges... received response... converting to JSON... writing binary output: debug/reports/problemranges.json... done @@ -30,6 +30,9 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null --timeout=.5s [cluster] retrieving SQL data for crdb_internal.cluster_transactions... writing output: debug/crdb_internal.cluster_transactions.txt... [cluster] retrieving SQL data for crdb_internal.cluster_transactions: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for crdb_internal.cluster_transactions: creating error output: debug/crdb_internal.cluster_transactions.txt.err.txt... done +[cluster] retrieving SQL data for crdb_internal.default_privileges... writing output: debug/crdb_internal.default_privileges.txt... +[cluster] retrieving SQL data for crdb_internal.default_privileges: last request failed: pq: query execution canceled due to statement timeout +[cluster] retrieving SQL data for crdb_internal.default_privileges: creating error output: debug/crdb_internal.default_privileges.txt.err.txt... done [cluster] retrieving SQL data for crdb_internal.jobs... writing output: debug/crdb_internal.jobs.txt... [cluster] retrieving SQL data for crdb_internal.jobs: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for crdb_internal.jobs: creating error output: debug/crdb_internal.jobs.txt.err.txt... done @@ -42,18 +45,27 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null --timeout=.5s [cluster] retrieving SQL data for system.namespace... writing output: debug/system.namespace.txt... [cluster] retrieving SQL data for system.namespace: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for system.namespace: creating error output: debug/system.namespace.txt.err.txt... done -[cluster] retrieving SQL data for system.namespace2... writing output: debug/system.namespace2.txt... -[cluster] retrieving SQL data for system.namespace2: last request failed: pq: query execution canceled due to statement timeout -[cluster] retrieving SQL data for system.namespace2: creating error output: debug/system.namespace2.txt.err.txt... done [cluster] retrieving SQL data for system.scheduled_jobs... writing output: debug/system.scheduled_jobs.txt... [cluster] retrieving SQL data for system.scheduled_jobs: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for system.scheduled_jobs: creating error output: debug/system.scheduled_jobs.txt.err.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_statements... writing output: debug/crdb_internal.create_statements.txt... +[cluster] retrieving SQL data for "".crdb_internal.create_statements: last request failed: pq: query execution canceled due to statement timeout +[cluster] retrieving SQL data for "".crdb_internal.create_statements: creating error output: debug/crdb_internal.create_statements.txt.err.txt... done +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements... writing output: debug/crdb_internal.create_type_statements.txt... +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements: last request failed: pq: query execution canceled due to statement timeout +[cluster] retrieving SQL data for "".crdb_internal.create_type_statements: creating error output: debug/crdb_internal.create_type_statements.txt.err.txt... done +[cluster] retrieving SQL data for crdb_internal.kv_node_liveness... writing output: debug/crdb_internal.kv_node_liveness.txt... +[cluster] retrieving SQL data for crdb_internal.kv_node_liveness: last request failed: pq: query execution canceled due to statement timeout +[cluster] retrieving SQL data for crdb_internal.kv_node_liveness: creating error output: debug/crdb_internal.kv_node_liveness.txt.err.txt... done [cluster] retrieving SQL data for crdb_internal.kv_node_status... writing output: debug/crdb_internal.kv_node_status.txt... [cluster] retrieving SQL data for crdb_internal.kv_node_status: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for crdb_internal.kv_node_status: creating error output: debug/crdb_internal.kv_node_status.txt.err.txt... done [cluster] retrieving SQL data for crdb_internal.kv_store_status... writing output: debug/crdb_internal.kv_store_status.txt... [cluster] retrieving SQL data for crdb_internal.kv_store_status: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for crdb_internal.kv_store_status: creating error output: debug/crdb_internal.kv_store_status.txt.err.txt... done +[cluster] retrieving SQL data for crdb_internal.regions... writing output: debug/crdb_internal.regions.txt... +[cluster] retrieving SQL data for crdb_internal.regions: last request failed: pq: query execution canceled due to statement timeout +[cluster] retrieving SQL data for crdb_internal.regions: creating error output: debug/crdb_internal.regions.txt.err.txt... done [cluster] retrieving SQL data for crdb_internal.schema_changes... writing output: debug/crdb_internal.schema_changes.txt... [cluster] retrieving SQL data for crdb_internal.schema_changes: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for crdb_internal.schema_changes: creating error output: debug/crdb_internal.schema_changes.txt.err.txt... done @@ -66,8 +78,11 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null --timeout=.5s [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... [cluster] retrieving SQL data for crdb_internal.invalid_objects: last request failed: pq: query execution canceled due to statement timeout [cluster] retrieving SQL data for crdb_internal.invalid_objects: creating error output: debug/crdb_internal.invalid_objects.txt.err.txt... done +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics: last request failed: pq: query execution canceled due to statement timeout +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics: creating error output: debug/crdb_internal.index_usage_statistics.txt.err.txt... done [cluster] requesting nodes... received response... -[cluster] requesting nodes: last request failed: operation "[cluster] requesting nodes" timed out after 500ms +[cluster] requesting nodes: last request failed: operation "[cluster] requesting nodes" timed out after 500ms: rpc error: ... [cluster] requesting nodes: creating error output: debug/nodes.json.err.txt... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done @@ -105,7 +120,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null --timeout=.5s [node 1] 1 log file ... [node 1] [log file ... [node 1] requesting ranges... received response... done -[node 1] 37 ranges found +[node 1] 42 ranges found [node 1] writing range 1... converting to JSON... writing binary output: debug/nodes/1/ranges/1.json... done [node 1] writing range 2... converting to JSON... writing binary output: debug/nodes/1/ranges/2.json... done [node 1] writing range 3... converting to JSON... writing binary output: debug/nodes/1/ranges/3.json... done @@ -143,13 +158,10 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null --timeout=.5s [node 1] writing range 35... converting to JSON... writing binary output: debug/nodes/1/ranges/35.json... done [node 1] writing range 36... converting to JSON... writing binary output: debug/nodes/1/ranges/36.json... done [node 1] writing range 37... converting to JSON... writing binary output: debug/nodes/1/ranges/37.json... done -[cluster] doctor examining cluster...... -[cluster] doctor examining cluster...: last request failed: query ' -SELECT id, descriptor, NULL AS mod_time_logical -FROM system.descriptor ORDER BY id': pq: query execution canceled due to statement timeout -[cluster] doctor examining cluster...: creating error output: debug/reports/doctor.txt.err.txt... done -[cluster] requesting list of SQL databases... received response... -[cluster] requesting list of SQL databases: last request failed: operation "[cluster] requesting list of SQL databases" timed out after 500ms -[cluster] requesting list of SQL databases: creating error output: debug/schema.err.txt... done +[node 1] writing range 38... converting to JSON... writing binary output: debug/nodes/1/ranges/38.json... done +[node 1] writing range 39... converting to JSON... writing binary output: debug/nodes/1/ranges/39.json... done +[node 1] writing range 40... converting to JSON... writing binary output: debug/nodes/1/ranges/40.json... done +[node 1] writing range 41... converting to JSON... writing binary output: debug/nodes/1/ranges/41.json... done +[node 1] writing range 42... converting to JSON... writing binary output: debug/nodes/1/ranges/42.json... done [cluster] pprof summary script... writing binary output: debug/pprof-summary.sh... done [cluster] hot range summary script... writing binary output: debug/hot-ranges.sh... done diff --git a/pkg/cli/zip.go b/pkg/cli/zip.go index 54bc9743404c..17427a31f1a8 100644 --- a/pkg/cli/zip.go +++ b/pkg/cli/zip.go @@ -246,11 +246,6 @@ func runDebugZip(cmd *cobra.Command, args []string) (retErr error) { return err } - // Collect the SQL schema. - if err := zc.collectSchemaData(ctx); err != nil { - return err - } - // Add a little helper script to draw attention to the existence of tags in // the profiles. { @@ -303,7 +298,7 @@ func (zc *debugZipContext) dumpTableDataForZip( zr *zipReporter, conn clisqlclient.Conn, base, table, query string, ) error { fullQuery := fmt.Sprintf(`SET statement_timeout = '%s'; %s`, zc.timeout, query) - baseName := base + "/" + table + baseName := base + "/" + sanitizeFilename(table) s := zr.start("retrieving SQL data for %s", table) const maxRetries = 5 @@ -347,3 +342,7 @@ func (zc *debugZipContext) dumpTableDataForZip( } return nil } + +func sanitizeFilename(f string) string { + return strings.TrimPrefix(f, `"".`) +} diff --git a/pkg/cli/zip_cluster_wide.go b/pkg/cli/zip_cluster_wide.go index 3f796f51ab85..f5e0751459e0 100644 --- a/pkg/cli/zip_cluster_wide.go +++ b/pkg/cli/zip_cluster_wide.go @@ -11,7 +11,6 @@ package cli import ( - "bytes" "context" "fmt" @@ -88,6 +87,12 @@ var debugZipTablesPerCluster = []string{ "system.namespace", "system.scheduled_jobs", + // The synthetic SQL CREATE statements for all tables. + // Note the "". to collect across all databases. + `"".crdb_internal.create_statements`, + // Ditto, for CREATE TYPE. + `"".crdb_internal.create_type_statements`, + "crdb_internal.kv_node_liveness", "crdb_internal.kv_node_status", "crdb_internal.kv_store_status", @@ -102,7 +107,6 @@ var debugZipTablesPerCluster = []string{ // collectClusterData runs the data collection that only needs to // occur once for the entire cluster. -// Also see collectSchemaData below. func (zc *debugZipContext) collectClusterData( ctx context.Context, firstNodeDetails *serverpb.DetailsResponse, ) (nodeList []statuspb.NodeStatus, livenessByNodeID nodeLivenesses, err error) { @@ -167,78 +171,3 @@ func (zc *debugZipContext) collectClusterData( return nodeList, livenessByNodeID, nil } - -// collectSchemaData collects the SQL logical schema once, for the entire cluster -// using the first node. This runs at the end, after all the per-node queries have -// been completed, because it has a higher likelihood to fail. -func (zc *debugZipContext) collectSchemaData(ctx context.Context) error { - // Run the debug doctor code over the schema. - { - var doctorData bytes.Buffer - s := zc.clusterPrinter.start("doctor examining cluster...") - descs, ns, jobs, doctorErr := fromCluster(zc.firstNodeSQLConn, zc.timeout) - if doctorErr == nil { - doctorErr = runDoctor("examine", descs, ns, jobs, &doctorData) - } - if err := zc.z.createRawOrError(s, reportsPrefix+"/doctor.txt", doctorData.Bytes(), doctorErr); err != nil { - return err - } - } - - // Collect the SQL schema. - { - var databases *serverpb.DatabasesResponse - s := zc.clusterPrinter.start("requesting list of SQL databases") - if err := zc.runZipFn(ctx, s, func(ctx context.Context) error { - var err error - databases, err = zc.admin.Databases(ctx, &serverpb.DatabasesRequest{}) - return err - }); err != nil { - if err := zc.z.createError(s, schemaPrefix, err); err != nil { - return err - } - } else { - s.done() - zc.clusterPrinter.info("%d databases found", len(databases.Databases)) - var dbEscaper fileNameEscaper - for _, dbName := range databases.Databases { - dbPrinter := zc.clusterPrinter.withPrefix("database: %s", dbName) - - prefix := schemaPrefix + "/" + dbEscaper.escape(dbName) - var database *serverpb.DatabaseDetailsResponse - s := dbPrinter.start("requesting database details") - requestErr := zc.runZipFn(ctx, s, - func(ctx context.Context) error { - var err error - database, err = zc.admin.DatabaseDetails(ctx, &serverpb.DatabaseDetailsRequest{Database: dbName}) - return err - }) - if err := zc.z.createJSONOrError(s, prefix+"@details.json", database, requestErr); err != nil { - return err - } - if requestErr != nil { - continue - } - - dbPrinter.info("%d tables found", len(database.TableNames)) - var tbEscaper fileNameEscaper - for _, tableName := range database.TableNames { - tbPrinter := dbPrinter.withPrefix("table: %s", tableName) - name := prefix + "/" + tbEscaper.escape(tableName) - var table *serverpb.TableDetailsResponse - s := tbPrinter.start("requesting table details") - requestErr := zc.runZipFn(ctx, s, - func(ctx context.Context) error { - var err error - table, err = zc.admin.TableDetails(ctx, &serverpb.TableDetailsRequest{Database: dbName, Table: tableName}) - return err - }) - if err := zc.z.createJSONOrError(s, name+".json", table, requestErr); err != nil { - return err - } - } - } - } - } - return nil -} diff --git a/pkg/cli/zip_helpers.go b/pkg/cli/zip_helpers.go index e4fbd8cb24db..004b8217cfdb 100644 --- a/pkg/cli/zip_helpers.go +++ b/pkg/cli/zip_helpers.go @@ -20,7 +20,6 @@ import ( "strconv" "strings" "time" - "unicode" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -138,42 +137,6 @@ func (z *zipper) createRawOrError(s *zipReporter, name string, b []byte, e error return z.createRaw(s, name, b) } -// fileNameEscaper is used to generate file names when the name of the -// file is derived from a SQL identifier or other stored data. This is -// necessary because not all characters in SQL identifiers and strings -// can be used in file names. -type fileNameEscaper struct { - counters map[string]int -} - -// escape ensures that f is stripped of characters that -// may be invalid in file names. The characters are also lowercased -// to ensure proper normalization in case-insensitive filesystems. -func (fne *fileNameEscaper) escape(f string) string { - f = strings.ToLower(f) - var out strings.Builder - for _, c := range f { - if c < 127 && (unicode.IsLetter(c) || unicode.IsDigit(c)) { - out.WriteRune(c) - } else { - out.WriteByte('_') - } - } - objName := out.String() - result := objName - - if fne.counters == nil { - fne.counters = make(map[string]int) - } - cnt := fne.counters[objName] - if cnt > 0 { - result += fmt.Sprintf("-%d", cnt) - } - cnt++ - fne.counters[objName] = cnt - return result -} - // nodeSelection is used to define a subset of the nodes on the command line. type nodeSelection struct { inclusive rangeSelection diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index a7e93dcc4f09..91bce7757ad0 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -71,8 +71,6 @@ table_name NOT IN ( 'cluster_contended_indexes', 'cluster_contended_tables', 'cluster_inflight_traces', - 'create_statements', - 'create_type_statements', 'cross_db_references', 'databases', 'forward_dependencies', @@ -109,7 +107,10 @@ ORDER BY name ASC`) var exp []string exp = append(exp, debugZipTablesPerNode...) - exp = append(exp, debugZipTablesPerCluster...) + for _, t := range debugZipTablesPerCluster { + t = strings.TrimPrefix(t, `"".`) + exp = append(exp, t) + } sort.Strings(exp) assert.Equal(t, exp, tables) @@ -325,10 +326,14 @@ func eraseNonDeterministicZipOutput(out string) string { out = re.ReplaceAllString(out, `rpc error: ...`) // The number of memory profiles previously collected is not deterministic. - re = regexp.MustCompile(`(?m)requesting heap files for node 1\.\.\..*found$`) - out = re.ReplaceAllString(out, `requesting heap files for node 1... ? found`) - re = regexp.MustCompile(`(?m)\^writing.*memprof*$`) + re = regexp.MustCompile(`(?m)^\[node \d+\] \d+ heap profiles found$`) + out = re.ReplaceAllString(out, `[node ?] ? heap profiles found`) + re = regexp.MustCompile(`(?m)^\[node \d+\] retrieving (memprof|memstats).*$` + "\n") + out = re.ReplaceAllString(out, ``) + re = regexp.MustCompile(`(?m)^\[node \d+\] writing profile.*$` + "\n") out = re.ReplaceAllString(out, ``) + + //out = strings.ReplaceAll(out, "\n\n", "\n") return out }