From d77f80e306045ded4115d5467adcae112a7f6565 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 29 Nov 2024 21:16:54 +0800 Subject: [PATCH 01/17] *: avoid notify privilege update for all users (#57042) ref pingcap/tidb#55563 --- .../restore/snap_client/systable_restore.go | 2 +- pkg/ddl/executor.go | 2 +- pkg/domain/BUILD.bazel | 1 + pkg/domain/domain.go | 102 +++++-- pkg/executor/grant.go | 3 +- pkg/executor/revoke.go | 11 +- pkg/executor/simple.go | 82 ++++-- .../passwordtest/password_management_test.go | 6 +- pkg/privilege/BUILD.bazel | 1 + pkg/privilege/privilege.go | 3 +- pkg/privilege/privileges/cache.go | 257 +++++++++++++----- pkg/privilege/privileges/privileges.go | 31 ++- pkg/server/tests/commontest/tidb_test.go | 2 +- pkg/session/session.go | 4 +- 14 files changed, 375 insertions(+), 132 deletions(-) diff --git a/br/pkg/restore/snap_client/systable_restore.go b/br/pkg/restore/snap_client/systable_restore.go index 55c40354d7307..de43f20784025 100644 --- a/br/pkg/restore/snap_client/systable_restore.go +++ b/br/pkg/restore/snap_client/systable_restore.go @@ -388,7 +388,7 @@ func (rc *SnapClient) afterSystemTablesReplaced(ctx context.Context, db string, var err error for _, table := range tables { if table == "user" { - if serr := rc.dom.NotifyUpdatePrivilege(); serr != nil { + if serr := rc.dom.NotifyUpdateAllUsersPrivilege(); serr != nil { log.Warn("failed to flush privileges, please manually execute `FLUSH PRIVILEGES`") err = multierr.Append(err, berrors.ErrUnknown.Wrap(serr).GenWithStack("failed to flush privileges")) } else { diff --git a/pkg/ddl/executor.go b/pkg/ddl/executor.go index a891f7fbeb7eb..6a715f744f2ec 100644 --- a/pkg/ddl/executor.go +++ b/pkg/ddl/executor.go @@ -6267,7 +6267,7 @@ func (e *executor) DropResourceGroup(ctx sessionctx.Context, stmt *ast.DropResou if checker == nil { return errors.New("miss privilege checker") } - user, matched := checker.MatchUserResourceGroupName(groupName.L) + user, matched := checker.MatchUserResourceGroupName(ctx.GetRestrictedSQLExecutor(), groupName.L) if matched { err = errors.Errorf("user [%s] depends on the resource group to drop", user) return err diff --git a/pkg/domain/BUILD.bazel b/pkg/domain/BUILD.bazel index 340bd723c1ad9..5c485aa4f452d 100644 --- a/pkg/domain/BUILD.bazel +++ b/pkg/domain/BUILD.bazel @@ -95,6 +95,7 @@ go_library( "//pkg/util/printer", "//pkg/util/replayer", "//pkg/util/servermemorylimit", + "//pkg/util/size", "//pkg/util/sqlexec", "//pkg/util/sqlkiller", "//pkg/util/syncutil", diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 2aa4e69648d0c..08de0973979d0 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -16,6 +16,7 @@ package domain import ( "context" + "encoding/json" "fmt" "math" "math/rand" @@ -98,6 +99,7 @@ import ( "github.com/pingcap/tidb/pkg/util/memoryusagealarm" "github.com/pingcap/tidb/pkg/util/replayer" "github.com/pingcap/tidb/pkg/util/servermemorylimit" + "github.com/pingcap/tidb/pkg/util/size" "github.com/pingcap/tidb/pkg/util/sqlkiller" "github.com/pingcap/tidb/pkg/util/syncutil" "github.com/tikv/client-go/v2/tikv" @@ -1885,6 +1887,36 @@ func (do *Domain) GetPDHTTPClient() pdhttp.Client { return nil } +func decodePrivilegeEvent(resp clientv3.WatchResponse) PrivilegeEvent { + var msg PrivilegeEvent + for _, event := range resp.Events { + if event.Kv != nil { + val := event.Kv.Value + if len(val) > 0 { + var tmp PrivilegeEvent + err := json.Unmarshal(val, &tmp) + if err != nil { + logutil.BgLogger().Warn("decodePrivilegeEvent unmarshal fail", zap.Error(err)) + break + } + if tmp.All { + msg.All = true + break + } + // duplicated users in list is ok. + msg.UserList = append(msg.UserList, tmp.UserList...) + } + } + } + + // In case old version triggers the event, the event value is empty, + // Then we fall back to the old way: reload all the users. + if len(msg.UserList) == 0 { + msg.All = true + } + return msg +} + // LoadPrivilegeLoop create a goroutine loads privilege tables in a loop, it // should be called only once in BootstrapSession. func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { @@ -1902,7 +1934,7 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { var watchCh clientv3.WatchChan duration := 5 * time.Minute if do.etcdClient != nil { - watchCh = do.etcdClient.Watch(context.Background(), privilegeKey) + watchCh = do.etcdClient.Watch(do.ctx, privilegeKey) duration = 10 * time.Minute } @@ -1914,25 +1946,32 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { var count int for { - ok := true + var err error select { case <-do.exit: return - case _, ok = <-watchCh: - case <-time.After(duration): - } - if !ok { - logutil.BgLogger().Warn("load privilege loop watch channel closed") - watchCh = do.etcdClient.Watch(context.Background(), privilegeKey) - count++ - if count > 10 { - time.Sleep(time.Duration(count) * time.Second) + case resp, ok := <-watchCh: + if ok { + count = 0 + event := decodePrivilegeEvent(resp) + if event.All { + err = do.privHandle.UpdateAll() + } else { + err = do.privHandle.Update(event.UserList) + } + } else { + if do.ctx.Err() == nil { + logutil.BgLogger().Error("load privilege loop watch channel closed") + watchCh = do.etcdClient.Watch(do.ctx, privilegeKey) + count++ + if count > 10 { + time.Sleep(time.Duration(count) * time.Second) + } + } } - continue + case <-time.After(duration): + err = do.privHandle.UpdateAll() } - - count = 0 - err := do.privHandle.Update() metrics.LoadPrivilegeCounter.WithLabelValues(metrics.RetLabel(err)).Inc() if err != nil { logutil.BgLogger().Warn("load privilege failed", zap.Error(err)) @@ -2923,15 +2962,39 @@ const ( tiflashComputeNodeKey = "/tiflash/new_tiflash_compute_nodes" ) +// PrivilegeEvent is the message definition for NotifyUpdatePrivilege(), encoded in json. +// TiDB old version do not use no such message. +type PrivilegeEvent struct { + All bool + UserList []string +} + +// NotifyUpdateAllUsersPrivilege updates privilege key in etcd, TiDB client that watches +// the key will get notification. +func (do *Domain) NotifyUpdateAllUsersPrivilege() error { + return do.notifyUpdatePrivilege(PrivilegeEvent{All: true}) +} + // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches // the key will get notification. -func (do *Domain) NotifyUpdatePrivilege() error { +func (do *Domain) NotifyUpdatePrivilege(userList []string) error { + return do.notifyUpdatePrivilege(PrivilegeEvent{UserList: userList}) +} + +func (do *Domain) notifyUpdatePrivilege(event PrivilegeEvent) error { // No matter skip-grant-table is configured or not, sending an etcd message is required. // Because we need to tell other TiDB instances to update privilege data, say, we're changing the // password using a special TiDB instance and want the new password to take effect. if do.etcdClient != nil { + data, err := json.Marshal(event) + if err != nil { + return errors.Trace(err) + } + if uint64(len(data)) > size.MB { + logutil.BgLogger().Warn("notify update privilege message too large", zap.ByteString("value", data)) + } row := do.etcdClient.KV - _, err := row.Put(context.Background(), privilegeKey, "") + _, err = row.Put(do.ctx, privilegeKey, string(data)) if err != nil { logutil.BgLogger().Warn("notify update privilege failed", zap.Error(err)) } @@ -2944,7 +3007,10 @@ func (do *Domain) NotifyUpdatePrivilege() error { return nil } - return do.PrivilegeHandle().Update() + if event.All { + return do.PrivilegeHandle().UpdateAll() + } + return do.PrivilegeHandle().Update(event.UserList) } // NotifyUpdateSysVarCache updates the sysvar cache key in etcd, which other TiDB diff --git a/pkg/executor/grant.go b/pkg/executor/grant.go index efe517b836685..9309be3224eb2 100644 --- a/pkg/executor/grant.go +++ b/pkg/executor/grant.go @@ -253,7 +253,8 @@ func (e *GrantExec) Next(ctx context.Context, _ *chunk.Chunk) error { return err } isCommit = true - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + users := userSpecToUserList(e.Users) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(users) } func containsNonDynamicPriv(privList []*ast.PrivElem) bool { diff --git a/pkg/executor/revoke.go b/pkg/executor/revoke.go index 79391c43bb7a8..ca631d08f6383 100644 --- a/pkg/executor/revoke.go +++ b/pkg/executor/revoke.go @@ -125,7 +125,16 @@ func (e *RevokeExec) Next(ctx context.Context, _ *chunk.Chunk) error { return err } isCommit = true - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + users := userSpecToUserList(e.Users) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(users) +} + +func userSpecToUserList(specs []*ast.UserSpec) []string { + users := make([]string, 0, len(specs)) + for _, user := range specs { + users = append(users, user.User.Username) + } + return users } // Checks that dynamic privileges are only of global scope. diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 9fa8f6450722e..403edfb7f28d9 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -409,6 +409,14 @@ func (e *SimpleExec) setDefaultRoleForCurrentUser(s *ast.SetDefaultRoleStmt) (er return nil } +func userIdentityToUserList(specs []*auth.UserIdentity) []string { + users := make([]string, 0, len(specs)) + for _, user := range specs { + users = append(users, user.Username) + } + return users +} + func (e *SimpleExec) executeSetDefaultRole(ctx context.Context, s *ast.SetDefaultRoleStmt) (err error) { sessionVars := e.Ctx().GetSessionVars() checker := privilege.GetPrivilegeManager(e.Ctx()) @@ -423,7 +431,8 @@ func (e *SimpleExec) executeSetDefaultRole(ctx context.Context, s *ast.SetDefaul if err != nil { return err } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + users := userIdentityToUserList(s.UserList) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(users) } } @@ -445,7 +454,8 @@ func (e *SimpleExec) executeSetDefaultRole(ctx context.Context, s *ast.SetDefaul if err != nil { return } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + users := userIdentityToUserList(s.UserList) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(users) } func (e *SimpleExec) setRoleRegular(s *ast.SetRoleStmt) error { @@ -758,7 +768,8 @@ func (e *SimpleExec) executeRevokeRole(ctx context.Context, s *ast.RevokeRoleStm u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + userList := userIdentityToUserList(s.Users) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(userList) } func (e *SimpleExec) executeCommit() { @@ -1265,7 +1276,8 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm if _, err := sqlExecutor.ExecuteInternal(internalCtx, "commit"); err != nil { return errors.Trace(err) } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + userList := userIdentityToUserList(users) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(userList) } func isRole(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, name, host string) (bool, error) { @@ -1770,7 +1782,7 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) } } - exists, err := userExistsInternal(ctx, sqlExecutor, spec.User.Username, spec.User.Hostname) + exists, _, err := userExistsInternal(ctx, sqlExecutor, spec.User.Username, spec.User.Hostname) if err != nil { return err } @@ -1779,6 +1791,10 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) failedUsers = append(failedUsers, user) continue } + currentAuthPlugin, err := privilege.GetPrivilegeManager(e.Ctx()).GetAuthPlugin(spec.User.Username, spec.User.Hostname) + if err != nil { + return err + } type AuthTokenOptionHandler int const ( @@ -1791,10 +1807,6 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) RequireAuthTokenOptions ) authTokenOptionHandler := noNeedAuthTokenOptions - currentAuthPlugin, err := privilege.GetPrivilegeManager(e.Ctx()).GetAuthPlugin(spec.User.Username, spec.User.Hostname) - if err != nil { - return err - } if currentAuthPlugin == mysql.AuthTiDBAuthToken { authTokenOptionHandler = OptionalAuthTokenOptions } @@ -2025,7 +2037,8 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) if _, err := sqlExecutor.ExecuteInternal(ctx, "commit"); err != nil { return err } - if err = domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(); err != nil { + users := userSpecToUserList(s.Specs) + if err = domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(users); err != nil { return err } if disableSandBoxMode { @@ -2101,7 +2114,8 @@ func (e *SimpleExec) executeGrantRole(ctx context.Context, s *ast.GrantRoleStmt) if _, err := sqlExecutor.ExecuteInternal(internalCtx, "commit"); err != nil { return err } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + userList := userIdentityToUserList(s.Users) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(userList) } // Should cover same internal mysql.* tables as DROP USER, so this function is very similar @@ -2126,7 +2140,7 @@ func (e *SimpleExec) executeRenameUser(s *ast.RenameUserStmt) error { if len(newUser.Hostname) > auth.HostNameMaxLength { return exeerrors.ErrWrongStringLength.GenWithStackByArgs(newUser.Hostname, "host name", auth.HostNameMaxLength) } - exists, err := userExistsInternal(ctx, sqlExecutor, oldUser.Username, oldUser.Hostname) + exists, _, err := userExistsInternal(ctx, sqlExecutor, oldUser.Username, oldUser.Hostname) if err != nil { return err } @@ -2135,7 +2149,7 @@ func (e *SimpleExec) executeRenameUser(s *ast.RenameUserStmt) error { break } - exists, err = userExistsInternal(ctx, sqlExecutor, newUser.Username, newUser.Hostname) + exists, _, err = userExistsInternal(ctx, sqlExecutor, newUser.Username, newUser.Hostname) if err != nil { return err } @@ -2216,7 +2230,13 @@ func (e *SimpleExec) executeRenameUser(s *ast.RenameUserStmt) error { if _, err := sqlExecutor.ExecuteInternal(ctx, "commit"); err != nil { return err } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + + userList := make([]string, 0, len(s.UserToUsers)*2) + for _, users := range s.UserToUsers { + userList = append(userList, users.OldUser.Username) + userList = append(userList, users.NewUser.Username) + } + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(userList) } func renameUserHostInSystemTable(sqlExecutor sqlexec.SQLExecutor, tableName, usernameColumn, hostColumn string, users *ast.UserToUser) error { @@ -2413,7 +2433,8 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) } } - return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + userList := userIdentityToUserList(s.UserList) + return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(userList) } func userExists(ctx context.Context, sctx sessionctx.Context, name string, host string) (bool, error) { @@ -2427,12 +2448,12 @@ func userExists(ctx context.Context, sctx sessionctx.Context, name string, host } // use the same internal executor to read within the same transaction, otherwise same as userExists -func userExistsInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, name string, host string) (bool, error) { +func userExistsInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, name string, host string) (bool, string, error) { sql := new(strings.Builder) sqlescape.MustFormatSQL(sql, `SELECT * FROM %n.%n WHERE User=%? AND Host=%? FOR UPDATE;`, mysql.SystemDB, mysql.UserTable, name, strings.ToLower(host)) recordSet, err := sqlExecutor.ExecuteInternal(ctx, sql.String()) if err != nil { - return false, err + return false, "", err } req := recordSet.NewChunk(nil) err = recordSet.Next(ctx, req) @@ -2440,11 +2461,27 @@ func userExistsInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, na if err == nil { rows = req.NumRows() } + + var authPlugin string + colIdx := -1 + for i, f := range recordSet.Fields() { + if f.ColumnAsName.L == "plugin" { + colIdx = i + } + } + if rows == 1 { + // rows can only be 0 or 1 + // When user + host does not exist, the rows is 0 + // When user + host exists, the rows is 1 because user + host is primary key of the table. + row := req.GetRow(0) + authPlugin = row.GetString(colIdx) + } + errClose := recordSet.Close() if errClose != nil { - return false, errClose + return false, "", errClose } - return rows > 0, err + return rows > 0, authPlugin, err } func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error { @@ -2481,13 +2518,14 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error h = s.User.Hostname checker := privilege.GetPrivilegeManager(e.Ctx()) + checker.MatchIdentity(u, h, false) activeRoles := e.Ctx().GetSessionVars().ActiveRoles if checker != nil && !checker.RequestVerification(activeRoles, "", "", "", mysql.SuperPriv) { currUser := e.Ctx().GetSessionVars().User return exeerrors.ErrDBaccessDenied.GenWithStackByArgs(currUser.Username, currUser.Hostname, "mysql") } } - exists, err := userExistsInternal(ctx, sqlExecutor, u, h) + exists, _, err := userExistsInternal(ctx, sqlExecutor, u, h) if err != nil { return err } @@ -2566,7 +2604,7 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error if _, err := sqlExecutor.ExecuteInternal(ctx, "commit"); err != nil { return err } - err = domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege() + err = domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege([]string{u}) if err != nil { return err } @@ -2698,7 +2736,7 @@ func (e *SimpleExec) executeFlush(s *ast.FlushStmt) error { } case ast.FlushPrivileges: dom := domain.GetDomain(e.Ctx()) - return dom.NotifyUpdatePrivilege() + return dom.NotifyUpdateAllUsersPrivilege() case ast.FlushTiDBPlugin: dom := domain.GetDomain(e.Ctx()) for _, pluginName := range s.Plugins { diff --git a/pkg/executor/test/passwordtest/password_management_test.go b/pkg/executor/test/passwordtest/password_management_test.go index 1a2e04dde5b45..1d66178004d54 100644 --- a/pkg/executor/test/passwordtest/password_management_test.go +++ b/pkg/executor/test/passwordtest/password_management_test.go @@ -218,7 +218,7 @@ func TestPasswordManagement(t *testing.T) { rootTK.MustExec(`set global validate_password.enable = OFF`) rootTK.MustExec(`update mysql.user set Password_last_changed = date_sub(Password_last_changed,interval '3 0:0:1' DAY_SECOND) where user = 'u2' and host = '%'`) - err = domain.GetDomain(rootTK.Session()).NotifyUpdatePrivilege() + err = domain.GetDomain(rootTK.Session()).NotifyUpdateAllUsersPrivilege() require.NoError(t, err) // Password expires and takes effect. err = tk.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "%"}, sha1Password("Uu3@22222"), nil, nil) @@ -723,7 +723,7 @@ func TestFailedLoginTrackingAlterUser(t *testing.T) { "JSON_EXTRACT(user_attributes, '$.Password_locking.failed_login_count')," + "JSON_EXTRACT(user_attributes, '$.Password_locking.password_lock_time_days')," + "JSON_EXTRACT(user_attributes, '$.metadata')from mysql.user where user= %? and host = %?" - err := domain.GetDomain(rootTK.Session()).NotifyUpdatePrivilege() + err := domain.GetDomain(rootTK.Session()).NotifyUpdateAllUsersPrivilege() require.NoError(t, err) rootTK.MustExec(`CREATE USER test1 IDENTIFIED BY '1234' FAILED_LOGIN_ATTEMPTS 3 PASSWORD_LOCK_TIME 3 COMMENT 'test'`) err = tk.Session().Auth(&auth.UserIdentity{Username: "test1", Hostname: "%"}, sha1Password("1234"), nil, nil) @@ -924,7 +924,7 @@ func changeAutoLockedLastChanged(tk *testkit.TestKit, ds, user string) { changeTime := time.Now().Add(d).Format(time.UnixDate) SQL = fmt.Sprintf(SQL, changeTime, user) tk.MustExec(SQL) - domain.GetDomain(tk.Session()).NotifyUpdatePrivilege() + domain.GetDomain(tk.Session()).NotifyUpdateAllUsersPrivilege() } func checkUserUserAttributes(tk *testkit.TestKit, user, host, row string) { diff --git a/pkg/privilege/BUILD.bazel b/pkg/privilege/BUILD.bazel index 2a8e6e17cba93..3e07babb15194 100644 --- a/pkg/privilege/BUILD.bazel +++ b/pkg/privilege/BUILD.bazel @@ -12,5 +12,6 @@ go_library( "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/types", + "//pkg/util/sqlexec", ], ) diff --git a/pkg/privilege/privilege.go b/pkg/privilege/privilege.go index cba7c122a7419..1591700e2238d 100644 --- a/pkg/privilege/privilege.go +++ b/pkg/privilege/privilege.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/sqlexec" ) type keyType int @@ -91,7 +92,7 @@ type Manager interface { MatchIdentity(user, host string, skipNameResolve bool) (string, string, bool) // MatchUserResourceGroupName matches a user with specified resource group name - MatchUserResourceGroupName(resourceGroupName string) (string, bool) + MatchUserResourceGroupName(exec sqlexec.RestrictedSQLExecutor, resourceGroupName string) (string, bool) // DBIsVisible returns true is the database is visible to current user. DBIsVisible(activeRole []*auth.RoleIdentity, db string) bool diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index b7f4492534c5f..0c82d7dce2f8b 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -347,6 +347,13 @@ func (p *MySQLPrivilege) FindRole(user string, host string, role *auth.RoleIdent return false } +func findRole(h *Handle, user string, host string, role *auth.RoleIdentity) bool { + terror.Log(h.ensureActiveUser(user)) + terror.Log(h.ensureActiveUser(role.Username)) + mysqlPrivilege := h.Get() + return mysqlPrivilege.FindRole(user, host, role) +} + // LoadAll loads the tables from database to memory. func (p *MySQLPrivilege) LoadAll(ctx sqlexec.RestrictedSQLExecutor) error { err := p.LoadUserTable(ctx) @@ -412,118 +419,162 @@ func (p *MySQLPrivilege) LoadAll(ctx sqlexec.RestrictedSQLExecutor) error { return nil } -func (p *immutable) loadSomeUsers(ctx sqlexec.RestrictedSQLExecutor, userList ...string) error { - err := p.loadTable(ctx, sqlLoadUserTable, p.decodeUserTableRow, userList...) +func findUserAndAllRoles(all map[string]struct{}, roleGraph map[string]roleGraphEdgesTable) { + for { + before := len(all) + + for userHost, value := range roleGraph { + user, _, found := strings.Cut(userHost, "@") + if !found { + // this should never happen + continue + } + if _, ok := all[user]; ok { + // If a user is in map, all its role should also added + for _, role := range value.roleList { + all[role.Username] = struct{}{} + } + } + } + + // loop until the map does not expand + after := len(all) + if before == after { + break + } + } +} + +func (p *immutable) loadSomeUsers(ctx sqlexec.RestrictedSQLExecutor, userList ...string) ([]string, error) { + // Load the full role edge table first. + p.roleGraph = make(map[string]roleGraphEdgesTable) + err := p.loadTable(ctx, sqlLoadRoleGraph, p.decodeRoleEdgesTable) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) + } + + // Including the user list and also their roles + extendedUserList := make(map[string]struct{}, len(userList)) + for _, user := range userList { + extendedUserList[user] = struct{}{} + } + findUserAndAllRoles(extendedUserList, p.roleGraph) + // Re-generate the user list. + userList = userList[:0] + for user := range extendedUserList { + userList = append(userList, user) + } + + err = p.loadTable(ctx, sqlLoadUserTable, p.decodeUserTableRow, userList...) + if err != nil { + return nil, errors.Trace(err) } err = p.loadTable(ctx, sqlLoadGlobalPrivTable, p.decodeGlobalPrivTableRow, userList...) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } err = p.loadTable(ctx, sqlLoadGlobalGrantsTable, p.decodeGlobalGrantsTableRow, userList...) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } err = p.loadTable(ctx, sqlLoadDBTable, p.decodeDBTableRow, userList...) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } err = p.loadTable(ctx, sqlLoadTablePrivTable, p.decodeTablesPrivTableRow, userList...) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } err = p.loadTable(ctx, sqlLoadDefaultRoles, p.decodeDefaultRoleTableRow, userList...) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } err = p.loadTable(ctx, sqlLoadColumnsPrivTable, p.decodeColumnsPrivTableRow, userList...) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } - p.roleGraph = make(map[string]roleGraphEdgesTable) - err = p.loadTable(ctx, sqlLoadRoleGraph, p.decodeRoleEdgesTable) - if err != nil { - return errors.Trace(err) - } - - return nil -} - -func dedupSortedKeepLast[S ~[]E, E any](s S, eq func(a, b E) bool) S { - skip := 0 - for i := 1; i < len(s); i++ { - if eq(s[i], s[i-1]) { - skip++ - } - s[i-skip] = s[i] - } - s = s[:len(s)-skip] - return s + return userList, nil } // merge construct a new MySQLPrivilege by merging the data of the two objects;. -func (p *MySQLPrivilege) merge(diff *immutable) *MySQLPrivilege { +func (p *MySQLPrivilege) merge(diff *immutable, userList []string) *MySQLPrivilege { var ret MySQLPrivilege ret.user = make([]UserRecord, 0, len(p.user)+len(diff.user)) - ret.user = append(ret.user, p.user...) + for _, v := range p.user { + if !slices.ContainsFunc(userList, func(usr string) bool { return usr == v.User }) { + ret.user = append(ret.user, v) + } + } ret.user = append(ret.user, diff.user...) - - // sort and dedup - slices.SortStableFunc(ret.user, compareUserRecord) - ret.user = dedupSortedKeepLast(ret.user, func(x, y UserRecord) bool { return x.User == y.User && x.Host == y.Host }) + slices.SortFunc(ret.user, compareUserRecord) ret.buildUserMap() ret.db = make([]dbRecord, 0, len(p.db)+len(diff.db)) - ret.db = append(ret.db, p.db...) + for _, v := range p.db { + if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { + ret.db = append(ret.db, v) + } + } ret.db = append(ret.db, diff.db...) + slices.SortFunc(ret.db, compareDBRecord) ret.buildDBMap() ret.tablesPriv = make([]tablesPrivRecord, 0, len(p.tablesPriv)+len(diff.tablesPriv)) - ret.tablesPriv = append(ret.tablesPriv, p.tablesPriv...) + for _, v := range p.tablesPriv { + if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { + ret.tablesPriv = append(ret.tablesPriv, v) + } + } ret.tablesPriv = append(ret.tablesPriv, diff.tablesPriv...) + slices.SortFunc(ret.tablesPriv, compareTablesPrivRecord) ret.buildTablesPrivMap() ret.columnsPriv = make([]columnsPrivRecord, 0, len(p.columnsPriv)+len(diff.columnsPriv)) - ret.columnsPriv = append(ret.columnsPriv, p.columnsPriv...) + for _, v := range p.columnsPriv { + if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { + ret.columnsPriv = append(ret.columnsPriv, v) + } + } ret.columnsPriv = append(ret.columnsPriv, diff.columnsPriv...) - slices.SortStableFunc(ret.columnsPriv, compareColumnsPrivRecord) - ret.columnsPriv = dedupSortedKeepLast(ret.columnsPriv, func(x, y columnsPrivRecord) bool { - return x.Host == y.Host && x.User == y.User && - x.DB == y.DB && x.TableName == y.TableName && x.ColumnName == y.ColumnName - }) + slices.SortFunc(ret.columnsPriv, compareColumnsPrivRecord) ret.defaultRoles = make([]defaultRoleRecord, 0, len(p.defaultRoles)+len(diff.defaultRoles)) - ret.defaultRoles = append(ret.defaultRoles, p.defaultRoles...) + for _, v := range p.defaultRoles { + if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { + ret.defaultRoles = append(ret.defaultRoles, v) + } + } ret.defaultRoles = append(ret.defaultRoles, diff.defaultRoles...) - slices.SortStableFunc(ret.defaultRoles, compareDefaultRoleRecord) - ret.defaultRoles = dedupSortedKeepLast(ret.defaultRoles, func(x, y defaultRoleRecord) bool { - return x.Host == y.Host && x.User == y.User - }) + slices.SortFunc(ret.defaultRoles, compareDefaultRoleRecord) ret.dynamicPriv = make([]dynamicPrivRecord, 0, len(p.dynamicPriv)+len(diff.dynamicPriv)) - ret.dynamicPriv = append(ret.dynamicPriv, p.dynamicPriv...) + for _, v := range p.dynamicPriv { + if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { + ret.dynamicPriv = append(ret.dynamicPriv, v) + } + } ret.dynamicPriv = append(ret.dynamicPriv, diff.dynamicPriv...) + slices.SortFunc(ret.dynamicPriv, compareDynamicPrivRecord) ret.buildDynamicMap() ret.globalPriv = make([]globalPrivRecord, 0, len(p.globalPriv)+len(diff.globalPriv)) - ret.globalPriv = append(ret.globalPriv, p.globalPriv...) + for _, v := range p.globalPriv { + if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { + ret.globalPriv = append(ret.globalPriv, v) + } + } ret.globalPriv = append(ret.globalPriv, diff.globalPriv...) - slices.SortStableFunc(ret.globalPriv, compareGlobalPrivRecord) - ret.globalPriv = dedupSortedKeepLast(ret.globalPriv, func(x, y globalPrivRecord) bool { - return x.Host == y.Host && x.User == y.User - }) + slices.SortFunc(ret.globalPriv, compareGlobalPrivRecord) ret.buildGlobalMap() ret.roleGraph = diff.roleGraph - return &ret } @@ -594,6 +645,10 @@ func compareGlobalPrivRecord(x, y globalPrivRecord) int { return compareBaseRecord(&x.baseRecord, &y.baseRecord) } +func compareDynamicPrivRecord(x, y dynamicPrivRecord) int { + return compareBaseRecord(&x.baseRecord, &y.baseRecord) +} + func compareColumnsPrivRecord(x, y columnsPrivRecord) int { cmp := compareBaseRecord(&x.baseRecord, &y.baseRecord) if cmp != 0 { @@ -711,7 +766,26 @@ func (p *MySQLPrivilege) LoadDBTable(ctx sqlexec.RestrictedSQLExecutor) error { } func compareDBRecord(x, y dbRecord) int { - return compareBaseRecord(&x.baseRecord, &y.baseRecord) + ret := compareBaseRecord(&x.baseRecord, &y.baseRecord) + if ret != 0 { + return ret + } + + return strings.Compare(x.DB, y.DB) +} + +func compareTablesPrivRecord(x, y tablesPrivRecord) int { + ret := compareBaseRecord(&x.baseRecord, &y.baseRecord) + if ret != 0 { + return ret + } + + ret = strings.Compare(x.DB, y.DB) + if ret != 0 { + return ret + } + + return strings.Compare(x.TableName, y.TableName) } func (p *MySQLPrivilege) buildDBMap() { @@ -1192,17 +1266,6 @@ func (p *MySQLPrivilege) matchIdentity(sctx sqlexec.RestrictedSQLExecutor, user, return nil } -// matchResoureGroup finds an identity to match resource group. -func (p *MySQLPrivilege) matchResoureGroup(resourceGroupName string) *UserRecord { - for i := 0; i < len(p.user); i++ { - record := &p.user[i] - if record.ResourceGroup == resourceGroupName { - return record - } - } - return nil -} - // connectionVerification verifies the username + hostname according to exact // match from the mysql.user privilege table. call matchIdentity() first if you // do not have an exact match yet. @@ -1834,22 +1897,74 @@ func NewHandle(sctx sqlexec.RestrictedSQLExecutor) *Handle { // ensureActiveUser ensure that the specific user data is loaded in-memory. func (h *Handle) ensureActiveUser(user string) error { + _, exist := h.activeUsers.Load(user) + if exist { + return nil + } + + var data immutable + userList, err := data.loadSomeUsers(h.sctx, user) + if err != nil { + return errors.Trace(err) + } + + h.merge(&data, userList) return nil } +func (h *Handle) merge(data *immutable, userList []string) { + for { + old := h.Get() + swapped := h.priv.CompareAndSwap(old, old.merge(data, userList)) + if swapped { + break + } + } + for _, user := range userList { + h.activeUsers.Store(user, struct{}{}) + } +} + // Get the MySQLPrivilege for read. func (h *Handle) Get() *MySQLPrivilege { return h.priv.Load() } -// Update loads all the privilege info from kv storage. -func (h *Handle) Update() error { - var priv MySQLPrivilege - err := priv.LoadAll(h.sctx) +// UpdateAll loads all the active users' privilege info from kv storage. +func (h *Handle) UpdateAll() error { + userList := make([]string, 0, 20) + h.activeUsers.Range(func(key, _ any) bool { + userList = append(userList, key.(string)) + return true + }) + + var priv immutable + userList, err := priv.loadSomeUsers(h.sctx, userList...) if err != nil { return err } + h.merge(&priv, userList) + return nil +} - h.priv.Store(&priv) +// Update loads the privilege info from kv storage for the list of users. +func (h *Handle) Update(userList []string) error { + needReload := false + for _, user := range userList { + if _, ok := h.activeUsers.Load(user); ok { + needReload = true + break + } + } + if !needReload { + return nil + } + + var priv immutable + userList, err := priv.loadSomeUsers(h.sctx, userList...) + if err != nil { + return err + } + h.merge(&priv, userList) return nil } diff --git a/pkg/privilege/privileges/privileges.go b/pkg/privilege/privileges/privileges.go index 5a607e56aa27c..b5aad336d080f 100644 --- a/pkg/privilege/privileges/privileges.go +++ b/pkg/privilege/privileges/privileges.go @@ -15,6 +15,7 @@ package privileges import ( + "context" "crypto/tls" "crypto/x509" "errors" @@ -29,6 +30,7 @@ import ( "github.com/lestrrat-go/jwx/v2/jwt/openid" "github.com/pingcap/tidb/pkg/extension" "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" @@ -43,6 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/sem" + "github.com/pingcap/tidb/pkg/util/sqlexec" "go.uber.org/zap" ) @@ -103,6 +106,7 @@ func (p *UserPrivileges) RequestDynamicVerificationWithUser(privName string, gra return false } + terror.Log(p.Handle.ensureActiveUser(user.Username)) mysqlPriv := p.Handle.Get() roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) return mysqlPriv.RequestDynamicVerification(roles, user.Username, user.Hostname, privName, grantable) @@ -315,6 +319,7 @@ func (p *UserPrivileges) isValidHash(record *UserRecord) bool { // GetEncodedPassword implements the Manager interface. func (p *UserPrivileges) GetEncodedPassword(user, host string) string { + terror.Log(p.Handle.ensureActiveUser(user)) mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { @@ -334,6 +339,7 @@ func (p *UserPrivileges) GetAuthPluginForConnection(user, host string) (string, return mysql.AuthNativePassword, nil } + terror.Log(p.Handle.ensureActiveUser(user)) mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { @@ -364,6 +370,8 @@ func (p *UserPrivileges) GetAuthPlugin(user, host string) (string, error) { if SkipWithGrant { return mysql.AuthNativePassword, nil } + + terror.Log(p.Handle.ensureActiveUser(user)) mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { @@ -393,11 +401,16 @@ func (p *UserPrivileges) MatchIdentity(user, host string, skipNameResolve bool) } // MatchUserResourceGroupName implements the Manager interface. -func (p *UserPrivileges) MatchUserResourceGroupName(resourceGroupName string) (u string, success bool) { - mysqlPriv := p.Handle.Get() - record := mysqlPriv.matchResoureGroup(resourceGroupName) - if record != nil { - return record.User, true +func (p *UserPrivileges) MatchUserResourceGroupName(exec sqlexec.RestrictedSQLExecutor, resourceGroupName string) (u string, success bool) { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnPrivilege) + sql := "SELECT user FROM mysql.user WHERE json_extract(user_attributes, '$.resource_group') = %? LIMIT 1" + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql, resourceGroupName) + if err != nil { + logutil.BgLogger().Error("execute sql error", zap.String("sql", sql), zap.Error(err)) + return "", false + } + if len(rows) > 0 { + return rows[0].GetString(0), true } return "", false } @@ -921,7 +934,6 @@ func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdent return nil, err } mysqlPrivilege := p.Handle.Get() - grants = mysqlPrivilege.showGrants(ctx, u, h, roles) if len(grants) == 0 { err = ErrNonexistingGrant.GenWithStackByArgs(u, h) @@ -935,11 +947,10 @@ func (p *UserPrivileges) ActiveRoles(ctx sessionctx.Context, roleList []*auth.Ro if SkipWithGrant { return true, "" } - mysqlPrivilege := p.Handle.Get() u := p.user h := p.host for _, r := range roleList { - ok := mysqlPrivilege.FindRole(u, h, r) + ok := findRole(p.Handle, u, h, r) if !ok { logutil.BgLogger().Error("find role failed", zap.Stringer("role", r)) return false, r.String() @@ -954,8 +965,7 @@ func (p *UserPrivileges) FindEdge(ctx sessionctx.Context, role *auth.RoleIdentit if SkipWithGrant { return false } - mysqlPrivilege := p.Handle.Get() - ok := mysqlPrivilege.FindRole(user.Username, user.Hostname, role) + ok := findRole(p.Handle, user.Username, user.Hostname, role) if !ok { logutil.BgLogger().Error("find role failed", zap.Stringer("role", role)) return false @@ -968,6 +978,7 @@ func (p *UserPrivileges) GetDefaultRoles(user, host string) []*auth.RoleIdentity if SkipWithGrant { return make([]*auth.RoleIdentity, 0, 10) } + terror.Log(p.Handle.ensureActiveUser(user)) mysqlPrivilege := p.Handle.Get() ret := mysqlPrivilege.getDefaultRoles(user, host) return ret diff --git a/pkg/server/tests/commontest/tidb_test.go b/pkg/server/tests/commontest/tidb_test.go index 3e8e28446ec64..8bc38e57af1d1 100644 --- a/pkg/server/tests/commontest/tidb_test.go +++ b/pkg/server/tests/commontest/tidb_test.go @@ -2666,7 +2666,7 @@ func TestSandBoxMode(t *testing.T) { require.NoError(t, err) _, err = Execute(context.Background(), qctx, "create user testuser;") require.NoError(t, err) - qctx.Session.GetSessionVars().User = &auth.UserIdentity{Username: "testuser", AuthUsername: "testuser", AuthHostname: "%"} + qctx.Session.Auth(&auth.UserIdentity{Username: "testuser", AuthUsername: "testuser", AuthHostname: "%"}, nil, nil, nil) alterPwdStmts := []string{ "set password = '1234';", diff --git a/pkg/session/session.go b/pkg/session/session.go index a511d8c5e7ae1..2ec07249815aa 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -2798,7 +2798,7 @@ func (s *session) Auth(user *auth.UserIdentity, authentication, salt []byte, aut } if lockStatusChanged { // Notification auto unlock. - err = domain.GetDomain(s).NotifyUpdatePrivilege() + err = domain.GetDomain(s).NotifyUpdatePrivilege([]string{authUser.Username}) if err != nil { return err } @@ -2972,7 +2972,7 @@ func authFailedTracking(s *session, user string, host string) (bool, *privileges func autolockAction(s *session, passwordLocking *privileges.PasswordLocking, user, host string) error { // Don't want to update the cache frequently, and only trigger the update cache when the lock status is updated. - err := domain.GetDomain(s).NotifyUpdatePrivilege() + err := domain.GetDomain(s).NotifyUpdatePrivilege([]string{user}) if err != nil { return err } From 75d0c65a7972ae2e7fdb65723b7d185e1621691c Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 26 Dec 2024 13:44:06 +0800 Subject: [PATCH 02/17] session: add index on 'user' field for mysql tables (#57470) ref pingcap/tidb#55563 --- .../snap_client/systable_restore_test.go | 2 +- pkg/domain/domain.go | 2 +- pkg/session/bootstrap.go | 50 ++++++++++++++----- .../bootstraptest/bootstrap_upgrade_test.go | 2 +- tests/integrationtest/r/explain.result | 25 ++++++++++ tests/integrationtest/t/explain.test | 7 +++ 6 files changed, 73 insertions(+), 15 deletions(-) diff --git a/br/pkg/restore/snap_client/systable_restore_test.go b/br/pkg/restore/snap_client/systable_restore_test.go index 5729ee1dd4e7c..45d1063389b98 100644 --- a/br/pkg/restore/snap_client/systable_restore_test.go +++ b/br/pkg/restore/snap_client/systable_restore_test.go @@ -393,7 +393,7 @@ func TestCheckPrivilegeTableRowsCollateCompatibility(t *testing.T) { // // The above variables are in the file br/pkg/restore/systable_restore.go func TestMonitorTheSystemTableIncremental(t *testing.T) { - require.Equal(t, int64(223), session.CurrentBootstrapVersion) + require.Equal(t, int64(224), session.CurrentBootstrapVersion) } func TestIsStatsTemporaryTable(t *testing.T) { diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 08de0973979d0..c167b97e605cf 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1927,7 +1927,7 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { return err } do.privHandle = privileges.NewHandle(sctx.GetRestrictedSQLExecutor()) - if err := do.privHandle.Update(); err != nil { + if err := do.privHandle.Update(nil); err != nil { return errors.Trace(err) } diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index f68458ee18b3f..2e775190d9635 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -115,14 +115,15 @@ const ( Password_expired ENUM('N','Y') NOT NULL DEFAULT 'N', Password_last_changed TIMESTAMP DEFAULT CURRENT_TIMESTAMP(), Password_lifetime SMALLINT UNSIGNED DEFAULT NULL, - PRIMARY KEY (Host, User));` + PRIMARY KEY (Host, User), + KEY i_user (User));` // CreateGlobalPrivTable is the SQL statement creates Global scope privilege table in system db. CreateGlobalPrivTable = "CREATE TABLE IF NOT EXISTS mysql.global_priv (" + "Host CHAR(255) NOT NULL DEFAULT ''," + "User CHAR(80) NOT NULL DEFAULT ''," + "Priv LONGTEXT NOT NULL DEFAULT ''," + - "PRIMARY KEY (Host, User)" + - ")" + "PRIMARY KEY (Host, User)," + + "KEY i_user (User))" // For `mysql.db`, `mysql.tables_priv` and `mysql.columns_priv` table, we have a slight different // schema definition with MySQL: columns `DB`/`Table_name`/`Column_name` are defined with case-insensitive @@ -160,7 +161,8 @@ const ( Execute_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Event_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Trigger_priv ENUM('N','Y') NOT NULL DEFAULT 'N', - PRIMARY KEY (Host, DB, User));` + PRIMARY KEY (Host, DB, User), + KEY i_user (User));` // CreateTablePrivTable is the SQL statement creates table scope privilege table in system db. CreateTablePrivTable = `CREATE TABLE IF NOT EXISTS mysql.tables_priv ( Host CHAR(255), @@ -171,7 +173,8 @@ const ( Timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, Table_priv SET('Select','Insert','Update','Delete','Create','Drop','Grant','Index','Alter','Create View','Show View','Trigger','References'), Column_priv SET('Select','Insert','Update','References'), - PRIMARY KEY (Host, DB, User, Table_name));` + PRIMARY KEY (Host, DB, User, Table_name), + KEY i_user (User));` // CreateColumnPrivTable is the SQL statement creates column scope privilege table in system db. CreateColumnPrivTable = `CREATE TABLE IF NOT EXISTS mysql.columns_priv( Host CHAR(255), @@ -181,7 +184,8 @@ const ( Column_name CHAR(64) CHARSET utf8mb4 COLLATE utf8mb4_general_ci, Timestamp TIMESTAMP DEFAULT CURRENT_TIMESTAMP, Column_priv SET('Select','Insert','Update','References'), - PRIMARY KEY (Host, DB, User, Table_name, Column_name));` + PRIMARY KEY (Host, DB, User, Table_name, Column_name), + KEY i_user (User));` // CreateGlobalVariablesTable is the SQL statement creates global variable table in system db. // TODO: MySQL puts GLOBAL_VARIABLES table in INFORMATION_SCHEMA db. // INFORMATION_SCHEMA is a virtual db in TiDB. So we put this table in system db. @@ -317,8 +321,8 @@ const ( USER CHAR(32) COLLATE utf8_bin NOT NULL DEFAULT '', DEFAULT_ROLE_HOST CHAR(60) COLLATE utf8_bin NOT NULL DEFAULT '%', DEFAULT_ROLE_USER CHAR(32) COLLATE utf8_bin NOT NULL DEFAULT '', - PRIMARY KEY (HOST,USER,DEFAULT_ROLE_HOST,DEFAULT_ROLE_USER) - )` + PRIMARY KEY (HOST,USER,DEFAULT_ROLE_HOST,DEFAULT_ROLE_USER), + KEY i_user (USER))` // CreateStatsTopNTable stores topn data of a cmsketch with top n. CreateStatsTopNTable = `CREATE TABLE IF NOT EXISTS mysql.stats_top_n ( @@ -380,7 +384,8 @@ const ( HOST char(255) NOT NULL DEFAULT '', PRIV char(32) NOT NULL DEFAULT '', WITH_GRANT_OPTION enum('N','Y') NOT NULL DEFAULT 'N', - PRIMARY KEY (USER,HOST,PRIV) + PRIMARY KEY (USER,HOST,PRIV), + KEY i_user (USER) );` // CreateCapturePlanBaselinesBlacklist stores the baseline capture filter rules. CreateCapturePlanBaselinesBlacklist = `CREATE TABLE IF NOT EXISTS mysql.capture_plan_baselines_blacklist ( @@ -1238,16 +1243,17 @@ const ( // add modify_params to tidb_global_task and tidb_global_task_history. version223 = 223 + // Add index on user field for some mysql tables. + version224 = 224 + // ... // [version223, version238] is the version range reserved for patches of 8.5.x // ... - - // next version should start with 239 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version223 +var currentBootstrapVersion int64 = version224 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -1421,11 +1427,17 @@ var ( upgradeToVer216, upgradeToVer217, upgradeToVer218, +<<<<<<< HEAD upgradeToVer219, upgradeToVer220, upgradeToVer221, upgradeToVer222, upgradeToVer223, +======= + upgradeToVer239, + upgradeToVer240, + upgradeToVer241, +>>>>>>> 8e1a47dddb (session: add index on 'user' field for mysql tables (#57470)) } ) @@ -3270,6 +3282,7 @@ func upgradeToVer219(s sessiontypes.Session, ver int64) { doReentrantDDL(s, addAnalyzeJobsSchemaTablePartitionStateIndex, dbterror.ErrDupKeyName) } +<<<<<<< HEAD func upgradeToVer220(s sessiontypes.Session, ver int64) { if ver >= version220 { return @@ -3302,6 +3315,19 @@ func upgradeToVer223(s sessiontypes.Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task_history ADD COLUMN modify_params json AFTER `error`;", infoschema.ErrColumnExists) } +func upgradeToVer224(s sessiontypes.Session, ver int64) { + if ver >= version224 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.user ADD INDEX i_user (user)", dbterror.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.global_priv ADD INDEX i_user (user)", dbterror.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.db ADD INDEX i_user (user)", dbterror.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.tables_priv ADD INDEX i_user (user)", dbterror.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.columns_priv ADD INDEX i_user (user)", dbterror.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.global_grants ADD INDEX i_user (user)", dbterror.ErrDupKeyName) + doReentrantDDL(s, "ALTER TABLE mysql.default_roles ADD INDEX i_user (user)", dbterror.ErrDupKeyName) +} + // initGlobalVariableIfNotExists initialize a global variable with specific val if it does not exist. func initGlobalVariableIfNotExists(s sessiontypes.Session, name string, val any) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) diff --git a/pkg/session/bootstraptest/bootstrap_upgrade_test.go b/pkg/session/bootstraptest/bootstrap_upgrade_test.go index 3070aeee5097a..d60d341966210 100644 --- a/pkg/session/bootstraptest/bootstrap_upgrade_test.go +++ b/pkg/session/bootstraptest/bootstrap_upgrade_test.go @@ -460,7 +460,7 @@ func TestUpgradeVersionForPausedJob(t *testing.T) { // checkDDLJobExecSucc is used to make sure the DDL operation is successful. func checkDDLJobExecSucc(t *testing.T, se sessiontypes.Session, jobID int64) { - sql := fmt.Sprintf(" admin show ddl jobs where job_id=%d", jobID) + sql := fmt.Sprintf(" admin show ddl jobs 20 where job_id=%d", jobID) suc := false for i := 0; i < 20; i++ { rows, err := execute(context.Background(), se, sql) diff --git a/tests/integrationtest/r/explain.result b/tests/integrationtest/r/explain.result index 6532a554d4fa0..520871ac7c81b 100644 --- a/tests/integrationtest/r/explain.result +++ b/tests/integrationtest/r/explain.result @@ -57,3 +57,28 @@ create view v as select cast(replace(substring_index(substring_index("",',',1),' desc v; Field Type Null Key Default Extra event_id varchar(32) NO NULL +explain format = 'brief' select * from mysql.user where user = 'xxx'; +id estRows task access object operator info +IndexLookUp 10.00 root +├─IndexRangeScan(Build) 10.00 cop[tikv] table:user, index:i_user(User) range:["xxx","xxx"], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 10.00 cop[tikv] table:user keep order:false, stats:pseudo +explain format = 'brief' select * from mysql.user where user = 'xxx' or user = 'yyy'; +id estRows task access object operator info +IndexLookUp 20.00 root +├─IndexRangeScan(Build) 20.00 cop[tikv] table:user, index:i_user(User) range:["xxx","xxx"], ["yyy","yyy"], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 20.00 cop[tikv] table:user keep order:false, stats:pseudo +explain format = 'brief' select * from mysql.global_priv where user = 'xxx'; +id estRows task access object operator info +IndexLookUp 10.00 root +├─IndexRangeScan(Build) 10.00 cop[tikv] table:global_priv, index:i_user(User) range:["xxx","xxx"], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 10.00 cop[tikv] table:global_priv keep order:false, stats:pseudo +explain format = 'brief' select * from mysql.global_grants where user = 'xxx' or user = 'yyy'; +id estRows task access object operator info +IndexLookUp 20.00 root +├─IndexRangeScan(Build) 20.00 cop[tikv] table:global_grants, index:i_user(USER) range:["xxx","xxx"], ["yyy","yyy"], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 20.00 cop[tikv] table:global_grants keep order:false, stats:pseudo +explain format = 'brief' select * from mysql.db where user = 'xxx'; +id estRows task access object operator info +IndexLookUp 10.00 root +├─IndexRangeScan(Build) 10.00 cop[tikv] table:db, index:i_user(User) range:["xxx","xxx"], keep order:false, stats:pseudo +└─TableRowIDScan(Probe) 10.00 cop[tikv] table:db keep order:false, stats:pseudo diff --git a/tests/integrationtest/t/explain.test b/tests/integrationtest/t/explain.test index ed679d54c199f..a1e63396e4504 100644 --- a/tests/integrationtest/t/explain.test +++ b/tests/integrationtest/t/explain.test @@ -21,3 +21,10 @@ drop table t; drop view if exists v; create view v as select cast(replace(substring_index(substring_index("",',',1),':',-1),'"','') as CHAR(32)) as event_id; desc v; + +# should use index lookup after adding user index, table scan is not expected +explain format = 'brief' select * from mysql.user where user = 'xxx'; +explain format = 'brief' select * from mysql.user where user = 'xxx' or user = 'yyy'; +explain format = 'brief' select * from mysql.global_priv where user = 'xxx'; +explain format = 'brief' select * from mysql.global_grants where user = 'xxx' or user = 'yyy'; +explain format = 'brief' select * from mysql.db where user = 'xxx'; \ No newline at end of file From 91b1a0c918ffea30eaf7ff15e7f21d8a99295116 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 11 Feb 2025 12:41:18 +0800 Subject: [PATCH 03/17] privilege/privileges: refactor the data struct for user privilege data (#58945) close pingcap/tidb#55563 --- pkg/domain/domain.go | 24 +- pkg/privilege/privileges/BUILD.bazel | 1 + pkg/privilege/privileges/cache.go | 613 +++++++++++------- pkg/privilege/privileges/cache_test.go | 74 ++- .../privileges/tidb_auth_token_test.go | 68 +- 5 files changed, 494 insertions(+), 286 deletions(-) diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index c167b97e605cf..43b7e6f13b942 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1946,19 +1946,14 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { var count int for { - var err error + var event PrivilegeEvent select { case <-do.exit: return case resp, ok := <-watchCh: if ok { count = 0 - event := decodePrivilegeEvent(resp) - if event.All { - err = do.privHandle.UpdateAll() - } else { - err = do.privHandle.Update(event.UserList) - } + event = decodePrivilegeEvent(resp) } else { if do.ctx.Err() == nil { logutil.BgLogger().Error("load privilege loop watch channel closed") @@ -1967,11 +1962,14 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { if count > 10 { time.Sleep(time.Duration(count) * time.Second) } + continue } } case <-time.After(duration): - err = do.privHandle.UpdateAll() + event.All = true } + + err := privReloadEvent(do.privHandle, &event) metrics.LoadPrivilegeCounter.WithLabelValues(metrics.RetLabel(err)).Inc() if err != nil { logutil.BgLogger().Warn("load privilege failed", zap.Error(err)) @@ -1981,6 +1979,16 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { return nil } +func privReloadEvent(h *privileges.Handle, event *PrivilegeEvent) (err error) { + switch { + case event.All: + err = h.UpdateAllActive() + default: + err = h.Update(event.UserList) + } + return +} + // LoadSysVarCacheLoop create a goroutine loads sysvar cache in a loop, // it should be called only once in BootstrapSession. func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { diff --git a/pkg/privilege/privileges/BUILD.bazel b/pkg/privilege/privileges/BUILD.bazel index b08fc60a8adac..76c30103e3311 100644 --- a/pkg/privilege/privileges/BUILD.bazel +++ b/pkg/privilege/privileges/BUILD.bazel @@ -35,6 +35,7 @@ go_library( "//pkg/util/sqlescape", "//pkg/util/sqlexec", "//pkg/util/stringutil", + "@com_github_google_btree//:btree", "@com_github_lestrrat_go_jwx_v2//jwk", "@com_github_lestrrat_go_jwx_v2//jws", "@com_github_lestrrat_go_jwx_v2//jwt", diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 0c82d7dce2f8b..759685db6b444 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -27,6 +27,7 @@ import ( "sync/atomic" "time" + "github.com/google/btree" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/auth" @@ -275,16 +276,85 @@ type immutable struct { roleGraph map[string]roleGraphEdgesTable } -type extended struct { - UserMap map[string][]UserRecord // Accelerate User searching - Global map[string][]globalPrivRecord - Dynamic map[string][]dynamicPrivRecord - DBMap map[string][]dbRecord // Accelerate DB searching - TablesPrivMap map[string][]tablesPrivRecord // Accelerate TablesPriv searching +type itemUser struct { + username string + data []UserRecord +} + +func compareItemUser(a, b itemUser) bool { + return a.username < b.username +} + +type itemDB struct { + username string + data []dbRecord +} + +func compareItemDB(a, b itemDB) bool { + return a.username < b.username +} + +type itemTablesPriv struct { + username string + data []tablesPrivRecord +} + +func compareItemTablesPriv(a, b itemTablesPriv) bool { + return a.username < b.username +} + +type itemColumnsPriv struct { + username string + data []columnsPrivRecord +} + +func compareItemColumnsPriv(a, b itemColumnsPriv) bool { + return a.username < b.username +} + +type itemDefaultRole struct { + username string + data []defaultRoleRecord +} + +func compareItemDefaultRole(a, b itemDefaultRole) bool { + return a.username < b.username +} + +type itemGlobalPriv struct { + username string + data []globalPrivRecord +} + +func compareItemGlobalPriv(a, b itemGlobalPriv) bool { + return a.username < b.username +} + +type itemDynamicPriv struct { + username string + data []dynamicPrivRecord +} + +func compareItemDynamicPriv(a, b itemDynamicPriv) bool { + return a.username < b.username +} + +type bTree[T any] struct { + *btree.BTreeG[T] + sync.Mutex +} + +// Clone provides the concurrent-safe operation by wraping the original Clone. +func (bt *bTree[T]) Clone() *btree.BTreeG[T] { + bt.Lock() + defer bt.Unlock() + return bt.BTreeG.Clone() } // MySQLPrivilege is the in-memory cache of mysql privilege tables. type MySQLPrivilege struct { + globalVars variable.GlobalVarAccessor + // In MySQL, a user identity consists of a user + host. // Either portion of user or host can contain wildcards, // requiring the privileges system to use a list-like @@ -294,9 +364,28 @@ type MySQLPrivilege struct { // which is that usernames can not contain wildcards. // This means that DB-records are organized in both a // slice (p.DB) and a Map (p.DBMap). - immutable - extended + user bTree[itemUser] + db bTree[itemDB] + tablesPriv bTree[itemTablesPriv] + columnsPriv bTree[itemColumnsPriv] + defaultRoles bTree[itemDefaultRole] + + globalPriv bTree[itemGlobalPriv] + dynamicPriv bTree[itemDynamicPriv] + roleGraph map[string]roleGraphEdgesTable +} + +func newMySQLPrivilege() *MySQLPrivilege { + var p MySQLPrivilege + p.user = bTree[itemUser]{BTreeG: btree.NewG(8, compareItemUser)} + p.db = bTree[itemDB]{BTreeG: btree.NewG(8, compareItemDB)} + p.tablesPriv = bTree[itemTablesPriv]{BTreeG: btree.NewG(8, compareItemTablesPriv)} + p.columnsPriv = bTree[itemColumnsPriv]{BTreeG: btree.NewG(8, compareItemColumnsPriv)} + p.defaultRoles = bTree[itemDefaultRole]{BTreeG: btree.NewG(8, compareItemDefaultRole)} + p.globalPriv = bTree[itemGlobalPriv]{BTreeG: btree.NewG(8, compareItemGlobalPriv)} + p.dynamicPriv = bTree[itemDynamicPriv]{BTreeG: btree.NewG(8, compareItemDynamicPriv)} + return &p } // FindAllUserEffectiveRoles is used to find all effective roles grant to this user. @@ -445,7 +534,10 @@ func findUserAndAllRoles(all map[string]struct{}, roleGraph map[string]roleGraph } } -func (p *immutable) loadSomeUsers(ctx sqlexec.RestrictedSQLExecutor, userList ...string) ([]string, error) { +func (p *MySQLPrivilege) loadSomeUsers(ctx sqlexec.RestrictedSQLExecutor, userList ...string) ([]string, error) { + if len(userList) > 512 { + logutil.BgLogger().Warn("loadSomeUsers called with a long user list", zap.Int("len", len(userList))) + } // Load the full role edge table first. p.roleGraph = make(map[string]roleGraphEdgesTable) err := p.loadTable(ctx, sqlLoadRoleGraph, p.decodeRoleEdgesTable) @@ -504,78 +596,87 @@ func (p *immutable) loadSomeUsers(ctx sqlexec.RestrictedSQLExecutor, userList .. } // merge construct a new MySQLPrivilege by merging the data of the two objects;. -func (p *MySQLPrivilege) merge(diff *immutable, userList []string) *MySQLPrivilege { - var ret MySQLPrivilege - ret.user = make([]UserRecord, 0, len(p.user)+len(diff.user)) - for _, v := range p.user { - if !slices.ContainsFunc(userList, func(usr string) bool { return usr == v.User }) { - ret.user = append(ret.user, v) - } - } - ret.user = append(ret.user, diff.user...) - slices.SortFunc(ret.user, compareUserRecord) - ret.buildUserMap() +func (p *MySQLPrivilege) merge(diff *MySQLPrivilege, userList []string) *MySQLPrivilege { + ret := newMySQLPrivilege() + user := p.user.Clone() + for _, u := range userList { + user.Delete(itemUser{username: u}) + } + diff.user.Ascend(func(itm itemUser) bool { + slices.SortFunc(itm.data, compareUserRecord) + user.ReplaceOrInsert(itm) + return true + }) + ret.user.BTreeG = user - ret.db = make([]dbRecord, 0, len(p.db)+len(diff.db)) - for _, v := range p.db { - if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { - ret.db = append(ret.db, v) - } + db := p.db.Clone() + for _, u := range userList { + db.Delete(itemDB{username: u}) } - ret.db = append(ret.db, diff.db...) - slices.SortFunc(ret.db, compareDBRecord) - ret.buildDBMap() + diff.db.Ascend(func(itm itemDB) bool { + slices.SortFunc(itm.data, compareDBRecord) + db.ReplaceOrInsert(itm) + return true + }) + ret.db.BTreeG = db - ret.tablesPriv = make([]tablesPrivRecord, 0, len(p.tablesPriv)+len(diff.tablesPriv)) - for _, v := range p.tablesPriv { - if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { - ret.tablesPriv = append(ret.tablesPriv, v) - } + tablesPriv := p.tablesPriv.Clone() + for _, u := range userList { + tablesPriv.Delete(itemTablesPriv{username: u}) } - ret.tablesPriv = append(ret.tablesPriv, diff.tablesPriv...) - slices.SortFunc(ret.tablesPriv, compareTablesPrivRecord) - ret.buildTablesPrivMap() + diff.tablesPriv.Ascend(func(itm itemTablesPriv) bool { + slices.SortFunc(itm.data, compareTablesPrivRecord) + tablesPriv.ReplaceOrInsert(itm) + return true + }) + ret.tablesPriv.BTreeG = tablesPriv - ret.columnsPriv = make([]columnsPrivRecord, 0, len(p.columnsPriv)+len(diff.columnsPriv)) - for _, v := range p.columnsPriv { - if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { - ret.columnsPriv = append(ret.columnsPriv, v) - } + columnsPriv := p.columnsPriv.Clone() + for _, u := range userList { + columnsPriv.Delete(itemColumnsPriv{username: u}) } - ret.columnsPriv = append(ret.columnsPriv, diff.columnsPriv...) - slices.SortFunc(ret.columnsPriv, compareColumnsPrivRecord) + diff.columnsPriv.Ascend(func(itm itemColumnsPriv) bool { + slices.SortFunc(itm.data, compareColumnsPrivRecord) + columnsPriv.ReplaceOrInsert(itm) + return true + }) + ret.columnsPriv.BTreeG = columnsPriv - ret.defaultRoles = make([]defaultRoleRecord, 0, len(p.defaultRoles)+len(diff.defaultRoles)) - for _, v := range p.defaultRoles { - if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { - ret.defaultRoles = append(ret.defaultRoles, v) - } + defaultRoles := p.defaultRoles.Clone() + for _, u := range userList { + defaultRoles.Delete(itemDefaultRole{username: u}) } - ret.defaultRoles = append(ret.defaultRoles, diff.defaultRoles...) - slices.SortFunc(ret.defaultRoles, compareDefaultRoleRecord) + diff.defaultRoles.Ascend(func(itm itemDefaultRole) bool { + slices.SortFunc(itm.data, compareDefaultRoleRecord) + defaultRoles.ReplaceOrInsert(itm) + return true + }) + ret.defaultRoles.BTreeG = defaultRoles - ret.dynamicPriv = make([]dynamicPrivRecord, 0, len(p.dynamicPriv)+len(diff.dynamicPriv)) - for _, v := range p.dynamicPriv { - if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { - ret.dynamicPriv = append(ret.dynamicPriv, v) - } + dynamicPriv := p.dynamicPriv.Clone() + for _, u := range userList { + dynamicPriv.Delete(itemDynamicPriv{username: u}) } - ret.dynamicPriv = append(ret.dynamicPriv, diff.dynamicPriv...) - slices.SortFunc(ret.dynamicPriv, compareDynamicPrivRecord) - ret.buildDynamicMap() + diff.dynamicPriv.Ascend(func(itm itemDynamicPriv) bool { + slices.SortFunc(itm.data, compareDynamicPrivRecord) + dynamicPriv.ReplaceOrInsert(itm) + return true + }) + ret.dynamicPriv.BTreeG = dynamicPriv - ret.globalPriv = make([]globalPrivRecord, 0, len(p.globalPriv)+len(diff.globalPriv)) - for _, v := range p.globalPriv { - if !slices.ContainsFunc(userList, func(user string) bool { return user == v.User }) { - ret.globalPriv = append(ret.globalPriv, v) - } + globalPriv := p.globalPriv.Clone() + for _, u := range userList { + globalPriv.Delete(itemGlobalPriv{username: u}) } - ret.globalPriv = append(ret.globalPriv, diff.globalPriv...) - slices.SortFunc(ret.globalPriv, compareGlobalPrivRecord) - ret.buildGlobalMap() + diff.globalPriv.Ascend(func(itm itemGlobalPriv) bool { + slices.SortFunc(itm.data, compareGlobalPrivRecord) + globalPriv.ReplaceOrInsert(itm) + return true + }) + ret.globalPriv.BTreeG = globalPriv ret.roleGraph = diff.roleGraph - return &ret + return ret } func noSuchTable(err error) bool { @@ -611,18 +712,9 @@ func (p *MySQLPrivilege) LoadUserTable(ctx sqlexec.RestrictedSQLExecutor) error // 3. The server uses the first row that matches the client host name and user name. // The server uses sorting rules that order rows with the most-specific Host values first. p.SortUserTable() - p.buildUserMap() return nil } -func (p *MySQLPrivilege) buildUserMap() { - userMap := make(map[string][]UserRecord, len(p.user)) - for _, record := range p.user { - userMap[record.User] = append(userMap[record.User], record) - } - p.UserMap = userMap -} - func compareBaseRecord(x, y *baseRecord) int { // Compare two item by user's host first. c1 := compareHost(x.Host, y.Host) @@ -725,16 +817,11 @@ func compareHost(x, y string) int { } // SortUserTable sorts p.User in the MySQLPrivilege struct. -func (p MySQLPrivilege) SortUserTable() { - slices.SortFunc(p.user, compareUserRecord) -} - -func (p *MySQLPrivilege) buildGlobalMap() { - global := make(map[string][]globalPrivRecord) - for _, value := range p.globalPriv { - global[value.User] = append(global[value.User], value) - } - p.Global = global +func (p *MySQLPrivilege) SortUserTable() { + p.user.Ascend(func(itm itemUser) bool { + slices.SortFunc(itm.data, compareUserRecord) + return true + }) } // LoadGlobalPrivTable loads the mysql.global_priv table from database. @@ -742,7 +829,6 @@ func (p *MySQLPrivilege) LoadGlobalPrivTable(ctx sqlexec.RestrictedSQLExecutor) if err := p.loadTable(ctx, sqlLoadGlobalPrivTable, p.decodeGlobalPrivTableRow); err != nil { return errors.Trace(err) } - p.buildGlobalMap() return nil } @@ -751,7 +837,6 @@ func (p *MySQLPrivilege) LoadGlobalGrantsTable(ctx sqlexec.RestrictedSQLExecutor if err := p.loadTable(ctx, sqlLoadGlobalGrantsTable, p.decodeGlobalGrantsTableRow); err != nil { return errors.Trace(err) } - p.buildDynamicMap() return nil } @@ -761,7 +846,6 @@ func (p *MySQLPrivilege) LoadDBTable(ctx sqlexec.RestrictedSQLExecutor) error { if err != nil { return err } - p.buildDBMap() return nil } @@ -788,45 +872,15 @@ func compareTablesPrivRecord(x, y tablesPrivRecord) int { return strings.Compare(x.TableName, y.TableName) } -func (p *MySQLPrivilege) buildDBMap() { - dbMap := make(map[string][]dbRecord, len(p.db)) - for _, record := range p.db { - dbMap[record.User] = append(dbMap[record.User], record) - } - - // Sort the records to make the matching rule work. - for _, records := range dbMap { - slices.SortFunc(records, compareDBRecord) - } - p.DBMap = dbMap -} - -func (p *MySQLPrivilege) buildDynamicMap() { - dynamic := make(map[string][]dynamicPrivRecord) - for _, value := range p.dynamicPriv { - dynamic[value.User] = append(dynamic[value.User], value) - } - p.Dynamic = dynamic -} - // LoadTablesPrivTable loads the mysql.tables_priv table from database. func (p *MySQLPrivilege) LoadTablesPrivTable(ctx sqlexec.RestrictedSQLExecutor) error { err := p.loadTable(ctx, sqlLoadTablePrivTable, p.decodeTablesPrivTableRow) if err != nil { return err } - p.buildTablesPrivMap() return nil } -func (p *MySQLPrivilege) buildTablesPrivMap() { - tablesPrivMap := make(map[string][]tablesPrivRecord, len(p.tablesPriv)) - for _, record := range p.tablesPriv { - tablesPrivMap[record.User] = append(tablesPrivMap[record.User], record) - } - p.TablesPrivMap = tablesPrivMap -} - // LoadColumnsPrivTable loads the mysql.columns_priv table from database. func (p *MySQLPrivilege) LoadColumnsPrivTable(ctx sqlexec.RestrictedSQLExecutor) error { return p.loadTable(ctx, sqlLoadColumnsPrivTable, p.decodeColumnsPrivTableRow) @@ -853,7 +907,7 @@ func addUserFilterCondition(sql string, userList []string) string { return b.String() } -func (p *immutable) loadTable(sctx sqlexec.RestrictedSQLExecutor, sql string, +func (p *MySQLPrivilege) loadTable(sctx sqlexec.RestrictedSQLExecutor, sql string, decodeTableRow func(chunk.Row, []*resolve.ResultField) error, userList ...string) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnPrivilege) sql = addUserFilterCondition(sql, userList) @@ -914,7 +968,7 @@ func (record *baseRecord) assignUserOrHost(row chunk.Row, i int, f *resolve.Resu } } -func (p *immutable) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value UserRecord for i, f := range fs { switch { @@ -998,11 +1052,16 @@ func (p *immutable) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) value.assignUserOrHost(row, i, f) } } - p.user = append(p.user, value) + old, ok := p.user.Get(itemUser{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.user.ReplaceOrInsert(old) return nil } -func (p *immutable) decodeGlobalPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeGlobalPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value globalPrivRecord for i, f := range fs { if f.ColumnAsName.L == "priv" { @@ -1032,11 +1091,16 @@ func (p *immutable) decodeGlobalPrivTableRow(row chunk.Row, fs []*resolve.Result value.assignUserOrHost(row, i, f) } } - p.globalPriv = append(p.globalPriv, value) + old, ok := p.globalPriv.Get(itemGlobalPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.globalPriv.ReplaceOrInsert(old) return nil } -func (p *immutable) decodeGlobalGrantsTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeGlobalGrantsTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value dynamicPrivRecord for i, f := range fs { switch f.ColumnAsName.L { @@ -1048,11 +1112,16 @@ func (p *immutable) decodeGlobalGrantsTableRow(row chunk.Row, fs []*resolve.Resu value.assignUserOrHost(row, i, f) } } - p.dynamicPriv = append(p.dynamicPriv, value) + old, ok := p.dynamicPriv.Get(itemDynamicPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.dynamicPriv.ReplaceOrInsert(old) return nil } -func (p *immutable) decodeDBTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeDBTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value dbRecord for i, f := range fs { switch { @@ -1072,11 +1141,16 @@ func (p *immutable) decodeDBTableRow(row chunk.Row, fs []*resolve.ResultField) e value.assignUserOrHost(row, i, f) } } - p.db = append(p.db, value) + old, ok := p.db.Get(itemDB{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.db.ReplaceOrInsert(old) return nil } -func (p *immutable) decodeTablesPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeTablesPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value tablesPrivRecord for i, f := range fs { switch f.ColumnAsName.L { @@ -1092,11 +1166,16 @@ func (p *immutable) decodeTablesPrivTableRow(row chunk.Row, fs []*resolve.Result value.assignUserOrHost(row, i, f) } } - p.tablesPriv = append(p.tablesPriv, value) + old, ok := p.tablesPriv.Get(itemTablesPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.tablesPriv.ReplaceOrInsert(old) return nil } -func (p *immutable) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.ResultField) error { var fromUser, fromHost, toHost, toUser string for i, f := range fs { switch f.ColumnAsName.L { @@ -1121,7 +1200,7 @@ func (p *immutable) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.ResultFiel return nil } -func (p *immutable) decodeDefaultRoleTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeDefaultRoleTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value defaultRoleRecord for i, f := range fs { switch f.ColumnAsName.L { @@ -1133,11 +1212,16 @@ func (p *immutable) decodeDefaultRoleTableRow(row chunk.Row, fs []*resolve.Resul value.assignUserOrHost(row, i, f) } } - p.defaultRoles = append(p.defaultRoles, value) + old, ok := p.defaultRoles.Get(itemDefaultRole{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.defaultRoles.ReplaceOrInsert(old) return nil } -func (p *immutable) decodeColumnsPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { +func (p *MySQLPrivilege) decodeColumnsPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value columnsPrivRecord for i, f := range fs { switch f.ColumnAsName.L { @@ -1159,7 +1243,12 @@ func (p *immutable) decodeColumnsPrivTableRow(row chunk.Row, fs []*resolve.Resul value.assignUserOrHost(row, i, f) } } - p.columnsPriv = append(p.columnsPriv, value) + old, ok := p.columnsPriv.Get(itemColumnsPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.columnsPriv.ReplaceOrInsert(old) return nil } @@ -1234,8 +1323,13 @@ func patternMatch(str string, patChars, patTypes []byte) bool { // matchIdentity finds an identity to match a user + host // using the correct rules according to MySQL. func (p *MySQLPrivilege) matchIdentity(sctx sqlexec.RestrictedSQLExecutor, user, host string, skipNameResolve bool) *UserRecord { - for i := 0; i < len(p.user); i++ { - record := &p.user[i] + item, ok := p.user.Get(itemUser{username: user}) + if !ok { + return nil + } + + for i := 0; i < len(item.data); i++ { + record := &item.data[i] if record.match(user, host) { return record } @@ -1255,8 +1349,8 @@ func (p *MySQLPrivilege) matchIdentity(sctx sqlexec.RestrictedSQLExecutor, user, return nil } for _, addr := range addrs { - for i := 0; i < len(p.user); i++ { - record := &p.user[i] + for i := 0; i < len(item.data); i++ { + record := &item.data[i] if record.match(user, addr) { return record } @@ -1270,10 +1364,10 @@ func (p *MySQLPrivilege) matchIdentity(sctx sqlexec.RestrictedSQLExecutor, user, // match from the mysql.user privilege table. call matchIdentity() first if you // do not have an exact match yet. func (p *MySQLPrivilege) connectionVerification(user, host string) *UserRecord { - records, exists := p.UserMap[user] + records, exists := p.user.Get(itemUser{username: user}) if exists { - for i := 0; i < len(records); i++ { - record := &records[i] + for i := 0; i < len(records.data); i++ { + record := &records.data[i] if record.Host == host { // exact match return record } @@ -1283,10 +1377,11 @@ func (p *MySQLPrivilege) connectionVerification(user, host string) *UserRecord { } func (p *MySQLPrivilege) matchGlobalPriv(user, host string) *globalPrivRecord { - uGlobal, exists := p.Global[user] + item, exists := p.globalPriv.Get(itemGlobalPriv{username: user}) if !exists { return nil } + uGlobal := item.data for i := 0; i < len(uGlobal); i++ { record := &uGlobal[i] if record.match(user, host) { @@ -1297,8 +1392,9 @@ func (p *MySQLPrivilege) matchGlobalPriv(user, host string) *globalPrivRecord { } func (p *MySQLPrivilege) matchUser(user, host string) *UserRecord { - records, exists := p.UserMap[user] + item, exists := p.user.Get(itemUser{username: user}) if exists { + records := item.data for i := 0; i < len(records); i++ { record := &records[i] if record.match(user, host) { @@ -1310,8 +1406,9 @@ func (p *MySQLPrivilege) matchUser(user, host string) *UserRecord { } func (p *MySQLPrivilege) matchDB(user, host, db string) *dbRecord { - records, exists := p.DBMap[user] + item, exists := p.db.Get(itemDB{username: user}) if exists { + records := item.data for i := 0; i < len(records); i++ { record := &records[i] if record.match(user, host, db) { @@ -1323,8 +1420,9 @@ func (p *MySQLPrivilege) matchDB(user, host, db string) *dbRecord { } func (p *MySQLPrivilege) matchTables(user, host, db, table string) *tablesPrivRecord { - records, exists := p.TablesPrivMap[user] + item, exists := p.tablesPriv.Get(itemTablesPriv{username: user}) if exists { + records := item.data for i := 0; i < len(records); i++ { record := &records[i] if record.match(user, host, db, table) { @@ -1336,10 +1434,13 @@ func (p *MySQLPrivilege) matchTables(user, host, db, table string) *tablesPrivRe } func (p *MySQLPrivilege) matchColumns(user, host, db, table, column string) *columnsPrivRecord { - for i := 0; i < len(p.columnsPriv); i++ { - record := &p.columnsPriv[i] - if record.match(user, host, db, table, column) { - return record + item, exists := p.columnsPriv.Get(itemColumnsPriv{username: user}) + if exists { + for i := 0; i < len(item.data); i++ { + record := &item.data[i] + if record.match(user, host, db, table, column) { + return record + } } } return nil @@ -1356,13 +1457,16 @@ func (p *MySQLPrivilege) HasExplicitlyGrantedDynamicPrivilege(activeRoles []*aut for _, r := range roleList { u := r.Username h := r.Hostname - for _, record := range p.Dynamic[u] { - if record.match(u, h) { - if withGrant && !record.GrantOption { - continue - } - if record.PrivilegeName == privName { - return true + item, exists := p.dynamicPriv.Get(itemDynamicPriv{username: u}) + if exists { + for _, record := range item.data { + if record.match(u, h) { + if withGrant && !record.GrantOption { + continue + } + if record.PrivilegeName == privName { + return true + } } } } @@ -1472,20 +1576,24 @@ func (p *MySQLPrivilege) DBIsVisible(user, host, db string) bool { } } - for _, record := range p.tablesPriv { - if record.baseRecord.match(user, host) && - strings.EqualFold(record.DB, db) { - if record.TablePriv != 0 || record.ColumnPriv != 0 { - return true + if item, exists := p.tablesPriv.Get(itemTablesPriv{username: user}); exists { + for _, record := range item.data { + if record.baseRecord.match(user, host) && + strings.EqualFold(record.DB, db) { + if record.TablePriv != 0 || record.ColumnPriv != 0 { + return true + } } } } - for _, record := range p.columnsPriv { - if record.baseRecord.match(user, host) && - strings.EqualFold(record.DB, db) { - if record.ColumnPriv != 0 { - return true + if item, exists := p.columnsPriv.Get(itemColumnsPriv{username: user}); exists { + for _, record := range item.data { + if record.baseRecord.match(user, host) && + strings.EqualFold(record.DB, db) { + if record.ColumnPriv != 0 { + return true + } } } } @@ -1504,10 +1612,12 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r var currentPriv mysql.PrivilegeType var userExists = false // Check whether user exists. - if userList, ok := p.UserMap[user]; ok { - for _, record := range userList { + if userList, ok := p.user.Get(itemUser{username: user}); ok { + for _, record := range userList.data { if record.fullyMatch(user, host) { userExists = true + hasGlobalGrant = true + currentPriv |= record.Privileges break } } @@ -1515,21 +1625,18 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r return gs } } - var g string - for _, record := range p.user { - if record.fullyMatch(user, host) { - hasGlobalGrant = true - currentPriv |= record.Privileges - } else { - for _, r := range allRoles { - if record.baseRecord.match(r.Username, r.Hostname) { + + for _, r := range allRoles { + if userList, ok := p.user.Get(itemUser{username: r.Username}); ok { + for _, record := range userList.data { + if record.fullyMatch(r.Username, r.Hostname) { hasGlobalGrant = true currentPriv |= record.Privileges } } } } - g = userPrivToString(currentPriv) + g := userPrivToString(currentPriv) if len(g) > 0 { var s string if (currentPriv & mysql.GrantPriv) > 0 { @@ -1554,17 +1661,20 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r // Show db scope grants. sortFromIdx = len(gs) dbPrivTable := make(map[string]mysql.PrivilegeType) - for _, record := range p.db { - if record.fullyMatch(user, host) { - dbPrivTable[record.DB] |= record.Privileges - } else { - for _, r := range allRoles { - if record.baseRecord.match(r.Username, r.Hostname) { - dbPrivTable[record.DB] |= record.Privileges + p.db.Ascend(func(itm itemDB) bool { + for _, record := range itm.data { + if record.fullyMatch(user, host) { + dbPrivTable[record.DB] |= record.Privileges + } else { + for _, r := range allRoles { + if record.baseRecord.match(r.Username, r.Hostname) { + dbPrivTable[record.DB] |= record.Privileges + } } } } - } + return true + }) sqlMode := ctx.GetSessionVars().SQLMode for dbName, priv := range dbPrivTable { @@ -1590,18 +1700,21 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r // Show table scope grants. sortFromIdx = len(gs) tablePrivTable := make(map[string]mysql.PrivilegeType) - for _, record := range p.tablesPriv { - recordKey := stringutil.Escape(record.DB, sqlMode) + "." + stringutil.Escape(record.TableName, sqlMode) - if user == record.User && host == record.Host { - tablePrivTable[recordKey] |= record.TablePriv - } else { - for _, r := range allRoles { - if record.baseRecord.match(r.Username, r.Hostname) { - tablePrivTable[recordKey] |= record.TablePriv + p.tablesPriv.Ascend(func(itm itemTablesPriv) bool { + for _, record := range itm.data { + recordKey := stringutil.Escape(record.DB, sqlMode) + "." + stringutil.Escape(record.TableName, sqlMode) + if user == record.User && host == record.Host { + tablePrivTable[recordKey] |= record.TablePriv + } else { + for _, r := range allRoles { + if record.baseRecord.match(r.Username, r.Hostname) { + tablePrivTable[recordKey] |= record.TablePriv + } } } } - } + return true + }) for k, priv := range tablePrivTable { g := tablePrivToString(priv) if len(g) > 0 { @@ -1625,14 +1738,16 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r // A map of "DB.Table" => Priv(col1, col2 ...) sortFromIdx = len(gs) columnPrivTable := make(map[string]privOnColumns) - for i := range p.columnsPriv { - record := p.columnsPriv[i] - if !collectColumnGrant(&record, user, host, columnPrivTable, sqlMode) { - for _, r := range allRoles { - collectColumnGrant(&record, r.Username, r.Hostname, columnPrivTable, sqlMode) + p.columnsPriv.Ascend(func(itm itemColumnsPriv) bool { + for _, record := range itm.data { + if !collectColumnGrant(&record, user, host, columnPrivTable, sqlMode) { + for _, r := range allRoles { + collectColumnGrant(&record, r.Username, r.Hostname, columnPrivTable, sqlMode) + } } } - } + return true + }) for k, v := range columnPrivTable { privCols := privOnColumnsToString(v) s := fmt.Sprintf(`GRANT %s ON %s TO '%s'@'%s'`, privCols, k, user, host) @@ -1667,21 +1782,25 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r // The convention is to merge the Dynamic privileges assigned to the user with // inherited dynamic privileges from those roles dynamicPrivsMap := make(map[string]bool) // privName, grantable - for _, record := range p.Dynamic[user] { - if record.fullyMatch(user, host) { - dynamicPrivsMap[record.PrivilegeName] = record.GrantOption + if item, exists := p.dynamicPriv.Get(itemDynamicPriv{username: user}); exists { + for _, record := range item.data { + if record.fullyMatch(user, host) { + dynamicPrivsMap[record.PrivilegeName] = record.GrantOption + } } } for _, r := range allRoles { - for _, record := range p.Dynamic[r.Username] { - if record.fullyMatch(r.Username, r.Hostname) { - // If the record already exists in the map and it's grantable - // skip doing anything, because we might inherit a non-grantable permission - // from a role, and don't want to clobber the existing privilege. - if grantable, ok := dynamicPrivsMap[record.PrivilegeName]; ok && grantable { - continue + if item, exists := p.dynamicPriv.Get(itemDynamicPriv{username: r.Username}); exists { + for _, record := range item.data { + if record.fullyMatch(r.Username, r.Hostname) { + // If the record already exists in the map and it's grantable + // skip doing anything, because we might inherit a non-grantable permission + // from a role, and don't want to clobber the existing privilege. + if grantable, ok := dynamicPrivsMap[record.PrivilegeName]; ok && grantable { + continue + } + dynamicPrivsMap[record.PrivilegeName] = record.GrantOption } - dynamicPrivsMap[record.PrivilegeName] = record.GrantOption } } } @@ -1802,18 +1921,22 @@ func (p *MySQLPrivilege) UserPrivilegesTable(activeRoles []*auth.RoleIdentity, u // This is verified against MySQL. showOtherUsers := p.RequestVerification(activeRoles, user, host, mysql.SystemDB, "", "", mysql.SelectPriv) var rows [][]types.Datum - for _, u := range p.user { - if showOtherUsers || u.match(user, host) { - rows = appendUserPrivilegesTableRow(rows, u) + p.user.Ascend(func(itm itemUser) bool { + for _, u := range itm.data { + if showOtherUsers || u.match(user, host) { + rows = appendUserPrivilegesTableRow(rows, u) + } } - } - for _, dynamicPrivs := range p.Dynamic { - for _, dynamicPriv := range dynamicPrivs { + return true + }) + p.dynamicPriv.Ascend(func(itm itemDynamicPriv) bool { + for _, dynamicPriv := range itm.data { if showOtherUsers || dynamicPriv.match(user, host) { rows = appendDynamicPrivRecord(rows, dynamicPriv) } } - } + return true + }) return rows } @@ -1857,9 +1980,11 @@ func appendUserPrivilegesTableRow(rows [][]types.Datum, user UserRecord) [][]typ func (p *MySQLPrivilege) getDefaultRoles(user, host string) []*auth.RoleIdentity { ret := make([]*auth.RoleIdentity, 0) - for _, r := range p.defaultRoles { - if r.match(user, host) { - ret = append(ret, &auth.RoleIdentity{Username: r.DefaultRoleUser, Hostname: r.DefaultRoleHost}) + if item, exists := p.defaultRoles.Get(itemDefaultRole{username: user}); exists { + for _, r := range item.data { + if r.match(user, host) { + ret = append(ret, &auth.RoleIdentity{Username: r.DefaultRoleUser, Hostname: r.DefaultRoleHost}) + } } } return ret @@ -1887,11 +2012,12 @@ type Handle struct { } // NewHandle returns a Handle. -func NewHandle(sctx sqlexec.RestrictedSQLExecutor) *Handle { - var priv MySQLPrivilege +func NewHandle(sctx sqlexec.RestrictedSQLExecutor, globalVars variable.GlobalVarAccessor) *Handle { + priv := newMySQLPrivilege() ret := &Handle{} ret.sctx = sctx - ret.priv.Store(&priv) + ret.globalVars = globalVars + ret.priv.Store(priv) return ret } @@ -1901,18 +2027,18 @@ func (h *Handle) ensureActiveUser(user string) error { if exist { return nil } - - var data immutable + data := newMySQLPrivilege() + data.globalVars = h.globalVars userList, err := data.loadSomeUsers(h.sctx, user) if err != nil { return errors.Trace(err) } - h.merge(&data, userList) + h.merge(data, userList) return nil } -func (h *Handle) merge(data *immutable, userList []string) { +func (h *Handle) merge(data *MySQLPrivilege, userList []string) { for { old := h.Get() swapped := h.priv.CompareAndSwap(old, old.merge(data, userList)) @@ -1932,23 +2058,43 @@ func (h *Handle) Get() *MySQLPrivilege { // UpdateAll loads all the active users' privilege info from kv storage. func (h *Handle) UpdateAll() error { + logutil.BgLogger().Warn("update all called") + priv := newMySQLPrivilege() + err := priv.LoadAll(h.sctx) + if err != nil { + return errors.Trace(err) + } + h.priv.Store(priv) + return nil +} + +// UpdateAllActive loads all the active users' privilege info from kv storage. +func (h *Handle) UpdateAllActive() error { userList := make([]string, 0, 20) h.activeUsers.Range(func(key, _ any) bool { userList = append(userList, key.(string)) return true }) + if len(userList) > 1024 { + logutil.BgLogger().Warn("active user count > 1024, revert to update all", zap.Int("len", len(userList))) + return h.UpdateAll() + } - var priv immutable + priv := newMySQLPrivilege() + priv.globalVars = h.globalVars userList, err := priv.loadSomeUsers(h.sctx, userList...) if err != nil { return err } - h.merge(&priv, userList) + h.merge(priv, userList) return nil } // Update loads the privilege info from kv storage for the list of users. func (h *Handle) Update(userList []string) error { + if len(userList) > 100 { + logutil.BgLogger().Warn("update user list is long", zap.Int("len", len(userList))) + } needReload := false for _, user := range userList { if _, ok := h.activeUsers.Load(user); ok { @@ -1960,11 +2106,12 @@ func (h *Handle) Update(userList []string) error { return nil } - var priv immutable + priv := newMySQLPrivilege() + priv.globalVars = h.globalVars userList, err := priv.loadSomeUsers(h.sctx, userList...) if err != nil { return err } - h.merge(&priv, userList) + h.merge(priv, userList) return nil } diff --git a/pkg/privilege/privileges/cache_test.go b/pkg/privilege/privileges/cache_test.go index 3527d347aac55..c1504244d1406 100644 --- a/pkg/privilege/privileges/cache_test.go +++ b/pkg/privilege/privileges/cache_test.go @@ -34,7 +34,7 @@ func TestLoadUserTable(t *testing.T) { tk.MustExec("use mysql;") tk.MustExec("truncate table user;") - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.Len(t, p.User(), 0) @@ -48,9 +48,9 @@ func TestLoadUserTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.user (Host, User, password_expired, password_last_changed, password_lifetime) VALUES ("%", "root2", "Y", "2022-10-10 12:00:00", 3)`) tk.MustExec(`INSERT INTO mysql.user (Host, User, password_expired, password_last_changed) VALUES ("%", "root3", "N", "2022-10-10 12:00:00")`) - p = privileges.MySQLPrivilege{} + p = privileges.NewMySQLPrivilege() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) - require.Len(t, p.User(), len(p.UserMap)) + // require.Len(t, p.User(), len(p.UserMap)) user := p.User() require.Equal(t, "root", user[0].User) @@ -66,6 +66,15 @@ func TestLoadUserTable(t *testing.T) { require.Equal(t, false, user[6].PasswordExpired) require.Equal(t, time.Date(2022, 10, 10, 12, 0, 0, 0, time.Local), user[6].PasswordLastChanged) require.Equal(t, int64(-1), user[6].PasswordLifeTime) + + // test switching default auth plugin + for _, plugin := range []string{mysql.AuthNativePassword, mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password} { + p = privileges.NewMySQLPrivilege() + p.SetGlobalVarsAccessor(se.GetSessionVars().GlobalVarsAccessor) + require.NoError(t, se.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.DefaultAuthPlugin, plugin)) + require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.Equal(t, plugin, p.User()[0].AuthPlugin) + } } func TestLoadGlobalPrivTable(t *testing.T) { @@ -78,18 +87,19 @@ func TestLoadGlobalPrivTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.global_priv VALUES ("%", "tu", "{\"access\":0,\"plugin\":\"mysql_native_password\",\"ssl_type\":3, \"ssl_cipher\":\"cipher\",\"x509_subject\":\"\C=ZH1\", \"x509_issuer\":\"\C=ZH2\", \"san\":\"\IP:127.0.0.1, IP:1.1.1.1, DNS:pingcap.com, URI:spiffe://mesh.pingcap.com/ns/timesh/sa/me1\", \"password_last_changed\":1}")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadGlobalPrivTable(se.GetRestrictedSQLExecutor())) - require.Equal(t, `%`, p.Global["tu"][0].Host) - require.Equal(t, `tu`, p.Global["tu"][0].User) - require.Equal(t, privileges.SslTypeSpecified, p.Global["tu"][0].Priv.SSLType) - require.Equal(t, "C=ZH2", p.Global["tu"][0].Priv.X509Issuer) - require.Equal(t, "C=ZH1", p.Global["tu"][0].Priv.X509Subject) - require.Equal(t, "IP:127.0.0.1, IP:1.1.1.1, DNS:pingcap.com, URI:spiffe://mesh.pingcap.com/ns/timesh/sa/me1", p.Global["tu"][0].Priv.SAN) - require.Len(t, p.Global["tu"][0].Priv.SANs[util.IP], 2) - require.Equal(t, "pingcap.com", p.Global["tu"][0].Priv.SANs[util.DNS][0]) - require.Equal(t, "spiffe://mesh.pingcap.com/ns/timesh/sa/me1", p.Global["tu"][0].Priv.SANs[util.URI][0]) + val := p.GlobalPriv("tu")[0] + require.Equal(t, `%`, val.Host) + require.Equal(t, `tu`, val.User) + require.Equal(t, privileges.SslTypeSpecified, val.Priv.SSLType) + require.Equal(t, "C=ZH2", val.Priv.X509Issuer) + require.Equal(t, "C=ZH1", val.Priv.X509Subject) + require.Equal(t, "IP:127.0.0.1, IP:1.1.1.1, DNS:pingcap.com, URI:spiffe://mesh.pingcap.com/ns/timesh/sa/me1", val.Priv.SAN) + require.Len(t, val.Priv.SANs[util.IP], 2) + require.Equal(t, "pingcap.com", val.Priv.SANs[util.DNS][0]) + require.Equal(t, "spiffe://mesh.pingcap.com/ns/timesh/sa/me1", val.Priv.SANs[util.URI][0]) } func TestLoadDBTable(t *testing.T) { @@ -102,10 +112,10 @@ func TestLoadDBTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.db (Host, DB, User, Select_priv, Insert_priv, Update_priv, Delete_priv, Create_priv) VALUES ("%", "information_schema", "root", "Y", "Y", "Y", "Y", "Y")`) tk.MustExec(`INSERT INTO mysql.db (Host, DB, User, Drop_priv, Grant_priv, Index_priv, Alter_priv, Create_view_priv, Show_view_priv, Execute_priv) VALUES ("%", "mysql", "root1", "Y", "Y", "Y", "Y", "Y", "Y", "Y")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) - require.Len(t, p.DB(), len(p.DBMap)) + // require.Len(t, p.DB(), len(p.DBMap)) require.Equal(t, mysql.SelectPriv|mysql.InsertPriv|mysql.UpdatePriv|mysql.DeletePriv|mysql.CreatePriv, p.DB()[0].Privileges) require.Equal(t, mysql.DropPriv|mysql.GrantPriv|mysql.IndexPriv|mysql.AlterPriv|mysql.CreateViewPriv|mysql.ShowViewPriv|mysql.ExecutePriv, p.DB()[1].Privileges) @@ -120,11 +130,11 @@ func TestLoadTablesPrivTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.tables_priv VALUES ("%", "db", "user", "table", "grantor", "2017-01-04 16:33:42.235831", "Grant,Index,Alter", "Insert,Update")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadTablesPrivTable(se.GetRestrictedSQLExecutor())) tablesPriv := p.TablesPriv() - require.Len(t, tablesPriv, len(p.TablesPrivMap)) + // require.Len(t, tablesPriv, len(p.TablesPrivMap)) require.Equal(t, `%`, tablesPriv[0].Host) require.Equal(t, "db", tablesPriv[0].DB) @@ -144,7 +154,7 @@ func TestLoadColumnsPrivTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.columns_priv VALUES ("%", "db", "user", "table", "column", "2017-01-04 16:33:42.235831", "Insert,Update")`) tk.MustExec(`INSERT INTO mysql.columns_priv VALUES ("127.0.0.1", "db", "user", "table", "column", "2017-01-04 16:33:42.235831", "Select")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadColumnsPrivTable(se.GetRestrictedSQLExecutor())) columnsPriv := p.ColumnsPriv() @@ -166,7 +176,7 @@ func TestLoadDefaultRoleTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.default_roles VALUES ("%", "test_default_roles", "localhost", "r_1")`) tk.MustExec(`INSERT INTO mysql.default_roles VALUES ("%", "test_default_roles", "localhost", "r_2")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadDefaultRoles(se.GetRestrictedSQLExecutor())) require.Equal(t, `%`, p.DefaultRoles()[0].Host) @@ -185,7 +195,7 @@ func TestPatternMatch(t *testing.T) { tk.MustExec("USE MYSQL;") tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("10.0.%", "root", "Y", "Y")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "root", "10.0.1", "test", "", "", mysql.SelectPriv)) @@ -198,7 +208,7 @@ func TestPatternMatch(t *testing.T) { tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("", "root", "Y", "N")`) - p = privileges.MySQLPrivilege{} + p = privileges.NewMySQLPrivilege() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "root", "", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "notnull", "test", "", "", mysql.SelectPriv)) @@ -222,7 +232,7 @@ func TestHostMatch(t *testing.T) { tk.MustExec("USE MYSQL;") tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (HOST, USER, authentication_string, Select_priv, Shutdown_priv) VALUES ("172.0.0.0/255.0.0.0", "root", "", "Y", "Y")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "root", "172.0.0.1", "test", "", "", mysql.SelectPriv)) @@ -248,7 +258,7 @@ func TestHostMatch(t *testing.T) { for _, IPMask := range cases { sql := fmt.Sprintf(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("%s", "root", "Y", "Y")`, IPMask) tk.MustExec(sql) - p = privileges.MySQLPrivilege{} + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.False(t, p.RequestVerification(activeRoles, "root", "127.0.0.1", "test", "", "", mysql.SelectPriv), fmt.Sprintf("test case: %s", IPMask)) @@ -258,7 +268,7 @@ func TestHostMatch(t *testing.T) { // Netmask notation cannot be used for IPv6 addresses. tk.MustExec(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("2001:db8::/ffff:ffff::", "root", "Y", "Y")`) - p = privileges.MySQLPrivilege{} + p = privileges.NewMySQLPrivilege() require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) require.False(t, p.RequestVerification(activeRoles, "root", "2001:db8::1234", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "2001:db8::", "test", "", "", mysql.SelectPriv)) @@ -274,7 +284,7 @@ func TestCaseInsensitive(t *testing.T) { tk.MustExec("CREATE TABLE TCTrain.TCTrainOrder (id int);") tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.db VALUES ("127.0.0.1", "TCTrain", "genius", "Y", "Y", "Y", "Y", "Y", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N")`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) // DB and Table names are case-insensitive in MySQL. @@ -290,7 +300,7 @@ func TestLoadRoleGraph(t *testing.T) { tk.MustExec("use mysql;") tk.MustExec("truncate table user;") - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) require.Len(t, p.User(), 0) @@ -300,7 +310,7 @@ func TestLoadRoleGraph(t *testing.T) { tk.MustExec(`INSERT INTO mysql.role_edges (FROM_HOST, FROM_USER, TO_HOST, TO_USER) VALUES ("%", "r_3", "%", "user1")`) tk.MustExec(`INSERT INTO mysql.role_edges (FROM_HOST, FROM_USER, TO_HOST, TO_USER) VALUES ("%", "r_4", "%", "root")`) - p = privileges.MySQLPrivilege{} + p = privileges.NewMySQLPrivilege() require.NoError(t, p.LoadRoleGraph(se.GetRestrictedSQLExecutor())) graph := p.RoleGraph() require.True(t, graph["root@%"].Find("r_2", "%")) @@ -323,7 +333,7 @@ func TestRoleGraphBFS(t *testing.T) { tk.MustExec(`GRANT r_1 TO r_4;`) tk.MustExec(`GRANT r_5 TO r_3, r_6;`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadRoleGraph(se.GetRestrictedSQLExecutor())) @@ -357,7 +367,7 @@ func TestFindAllUserEffectiveRoles(t *testing.T) { tk.MustExec(`GRANT r_1 to u1`) tk.MustExec(`GRANT r_2 to u1`) - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadAll(se.GetRestrictedSQLExecutor())) ret := p.FindAllUserEffectiveRoles("u1", "%", []*auth.RoleIdentity{ @@ -382,7 +392,7 @@ func TestFindAllUserEffectiveRoles(t *testing.T) { } func TestSortUserTable(t *testing.T) { - var p privileges.MySQLPrivilege + p := privileges.NewMySQLPrivilege() p.SetUser([]privileges.UserRecord{ privileges.NewUserRecord(`%`, "root"), privileges.NewUserRecord(`%`, "jeffrey"), @@ -391,8 +401,8 @@ func TestSortUserTable(t *testing.T) { }) p.SortUserTable() result := []privileges.UserRecord{ - privileges.NewUserRecord("localhost", "root"), privileges.NewUserRecord("localhost", ""), + privileges.NewUserRecord("localhost", "root"), privileges.NewUserRecord(`%`, "jeffrey"), privileges.NewUserRecord(`%`, "root"), } @@ -453,7 +463,7 @@ func TestDBIsVisible(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("create database visdb") - p := privileges.MySQLPrivilege{} + p := privileges.NewMySQLPrivilege() se := tk.Session() require.NoError(t, p.LoadAll(se.GetRestrictedSQLExecutor())) diff --git a/pkg/privilege/privileges/tidb_auth_token_test.go b/pkg/privilege/privileges/tidb_auth_token_test.go index d03190bc7a15f..791a6a943e1eb 100644 --- a/pkg/privilege/privileges/tidb_auth_token_test.go +++ b/pkg/privilege/privileges/tidb_auth_token_test.go @@ -20,6 +20,7 @@ import ( "fmt" "log" "os" + "slices" "strings" "testing" "time" @@ -418,30 +419,71 @@ func TestJWKSImpl(t *testing.T) { require.Error(t, err) } -func (p *immutable) User() []UserRecord { - return p.user +func (p *MySQLPrivilege) User() []UserRecord { + var ret []UserRecord + p.user.Ascend(func(itm itemUser) bool { + ret = append(ret, itm.data...) + return true + }) + slices.SortStableFunc(ret, compareUserRecord) + return ret } -func (p *immutable) SetUser(user []UserRecord) { - p.user = user +func (p *MySQLPrivilege) SetUser(user []UserRecord) { + p.user.Clear(false) + for _, u := range user { + old, exists := p.user.Get(itemUser{username: u.User}) + if !exists { + old.username = u.User + } + old.data = append(old.data, u) + p.user.ReplaceOrInsert(old) + } } -func (p *immutable) DB() []dbRecord { - return p.db +func (p *MySQLPrivilege) DB() []dbRecord { + var ret []dbRecord + p.db.Ascend(func(itm itemDB) bool { + ret = append(ret, itm.data...) + return true + }) + return ret } -func (p *immutable) TablesPriv() []tablesPrivRecord { - return p.tablesPriv +func (p *MySQLPrivilege) TablesPriv() []tablesPrivRecord { + var ret []tablesPrivRecord + p.tablesPriv.Ascend(func(itm itemTablesPriv) bool { + ret = append(ret, itm.data...) + return true + }) + return ret } -func (p *immutable) ColumnsPriv() []columnsPrivRecord { - return p.columnsPriv +func (p *MySQLPrivilege) ColumnsPriv() []columnsPrivRecord { + var ret []columnsPrivRecord + p.columnsPriv.Ascend(func(itm itemColumnsPriv) bool { + ret = append(ret, itm.data...) + return true + }) + return ret } -func (p *immutable) DefaultRoles() []defaultRoleRecord { - return p.defaultRoles +func (p *MySQLPrivilege) DefaultRoles() []defaultRoleRecord { + var ret []defaultRoleRecord + p.defaultRoles.Ascend(func(itm itemDefaultRole) bool { + ret = append(ret, itm.data...) + return true + }) + return ret } -func (p *immutable) RoleGraph() map[string]roleGraphEdgesTable { +func (p *MySQLPrivilege) GlobalPriv(user string) []globalPrivRecord { + ret, _ := p.globalPriv.Get(itemGlobalPriv{username: user}) + return ret.data +} + +func (p *MySQLPrivilege) RoleGraph() map[string]roleGraphEdgesTable { return p.roleGraph } + +var NewMySQLPrivilege = newMySQLPrivilege From a8028eb2fbe86863b89f59cc0e7f76c5259e8ec6 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 12 Feb 2025 11:35:32 +0800 Subject: [PATCH 04/17] *: add sql variable @@tidb_accelerate_user_creation_update (#58512) ref pingcap/tidb#55563 --- pkg/domain/domain.go | 7 +- pkg/privilege/privileges/BUILD.bazel | 1 + pkg/privilege/privileges/cache.go | 22 +- pkg/privilege/privileges/privileges_test.go | 75 + .../privileges/tidb_auth_token_test.go | 4 + pkg/sessionctx/vardef/tidb_vars.go | 2020 +++++++++++++++++ pkg/sessionctx/variable/sysvar.go | 6 + 7 files changed, 2129 insertions(+), 6 deletions(-) create mode 100644 pkg/sessionctx/vardef/tidb_vars.go diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 43b7e6f13b942..51ee36002c49e 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1981,6 +1981,8 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { func privReloadEvent(h *privileges.Handle, event *PrivilegeEvent) (err error) { switch { + case !vardef.AccelerateUserCreationUpdate.Load(): + err = h.UpdateAll() case event.All: err = h.UpdateAllActive() default: @@ -3015,10 +3017,7 @@ func (do *Domain) notifyUpdatePrivilege(event PrivilegeEvent) error { return nil } - if event.All { - return do.PrivilegeHandle().UpdateAll() - } - return do.PrivilegeHandle().Update(event.UserList) + return privReloadEvent(do.PrivilegeHandle(), &event) } // NotifyUpdateSysVarCache updates the sysvar cache key in etcd, which other TiDB diff --git a/pkg/privilege/privileges/BUILD.bazel b/pkg/privilege/privileges/BUILD.bazel index 76c30103e3311..72e7353d8f1a6 100644 --- a/pkg/privilege/privileges/BUILD.bazel +++ b/pkg/privilege/privileges/BUILD.bazel @@ -59,6 +59,7 @@ go_test( shard_count = 50, deps = [ "//pkg/config", + "//pkg/domain", "//pkg/errno", "//pkg/kv", "//pkg/parser/auth", diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 759685db6b444..48b521e7a1425 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -846,6 +846,10 @@ func (p *MySQLPrivilege) LoadDBTable(ctx sqlexec.RestrictedSQLExecutor) error { if err != nil { return err } + p.db.Ascend(func(itm itemDB) bool { + slices.SortFunc(itm.data, compareDBRecord) + return true + }) return nil } @@ -2009,6 +2013,8 @@ type Handle struct { // Only load the active user's data to save memory // username => struct{} activeUsers sync.Map + fullData atomic.Bool + globalVars variable.GlobalVarAccessor } // NewHandle returns a Handle. @@ -2022,7 +2028,16 @@ func NewHandle(sctx sqlexec.RestrictedSQLExecutor, globalVars variable.GlobalVar } // ensureActiveUser ensure that the specific user data is loaded in-memory. -func (h *Handle) ensureActiveUser(user string) error { +func (h *Handle) ensureActiveUser(ctx context.Context, user string) error { + if p := ctx.Value("mock"); p != nil { + visited := p.(*bool) + *visited = true + } + if h.fullData.Load() { + // All users data are in-memory, nothing to do + return nil + } + _, exist := h.activeUsers.Load(user) if exist { return nil @@ -2056,7 +2071,7 @@ func (h *Handle) Get() *MySQLPrivilege { return h.priv.Load() } -// UpdateAll loads all the active users' privilege info from kv storage. +// UpdateAll loads all the users' privilege info from kv storage. func (h *Handle) UpdateAll() error { logutil.BgLogger().Warn("update all called") priv := newMySQLPrivilege() @@ -2065,11 +2080,13 @@ func (h *Handle) UpdateAll() error { return errors.Trace(err) } h.priv.Store(priv) + h.fullData.Store(true) return nil } // UpdateAllActive loads all the active users' privilege info from kv storage. func (h *Handle) UpdateAllActive() error { + h.fullData.Store(false) userList := make([]string, 0, 20) h.activeUsers.Range(func(key, _ any) bool { userList = append(userList, key.(string)) @@ -2092,6 +2109,7 @@ func (h *Handle) UpdateAllActive() error { // Update loads the privilege info from kv storage for the list of users. func (h *Handle) Update(userList []string) error { + h.fullData.Store(false) if len(userList) > 100 { logutil.BgLogger().Warn("update user list is long", zap.Int("len", len(userList))) } diff --git a/pkg/privilege/privileges/privileges_test.go b/pkg/privilege/privileges/privileges_test.go index e483690852c26..9fd80b0176d16 100644 --- a/pkg/privilege/privileges/privileges_test.go +++ b/pkg/privilege/privileges/privileges_test.go @@ -30,6 +30,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/auth" @@ -2116,3 +2117,77 @@ func TestShowGrantsSQLMode(t *testing.T) { "GRANT SELECT ON \"test\".* TO 'show_sql_mode'@'localhost'", }) } + +func TestEnsureActiveUserCoverage(t *testing.T) { + store := createStoreAndPrepareDB(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create user 'test'") + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil, nil) + + cases := []struct { + sql string + visited bool + }{ + {"drop user if exists 'test1'", false}, + {"alter user test identified by 'test1'", false}, + {"set password for test = 'test2'", false}, + {"show create user test", false}, + {"create user test1", false}, + {"grant select on test.* to test1", false}, + {"show grants", true}, + {"show grants for 'test'@'%'", true}, + } + + for ith, c := range cases { + var visited bool + ctx := context.WithValue(context.Background(), "mock", &visited) + rs, err := tk.ExecWithContext(ctx, c.sql) + require.NoError(t, err) + + comment := fmt.Sprintf("testcase %d failed", ith) + if rs != nil { + tk.ResultSetToResultWithCtx(ctx, rs, comment) + } + require.Equal(t, c.visited, visited, comment) + } +} + +func TestSQLVariableAccelerateUserCreationUpdate(t *testing.T) { + store := createStoreAndPrepareDB(t) + tk := testkit.NewTestKit(t, store) + dom := domain.GetDomain(tk.Session()) + // 1. check the default variable value + tk.MustQuery("select @@global.tidb_accelerate_user_creation_update").Check(testkit.Rows("0")) + // trigger priv reload + tk.MustExec("create user aaa") + handle := dom.PrivilegeHandle() + handle.CheckFullData(t, true) + priv := handle.Get() + require.False(t, priv.RequestVerification(nil, "bbb", "%", "test", "", "", mysql.SelectPriv)) + + // 2. change the variable and check + tk.MustExec("set @@global.tidb_accelerate_user_creation_update = on") + tk.MustQuery("select @@global.tidb_accelerate_user_creation_update").Check(testkit.Rows("1")) + require.True(t, vardef.AccelerateUserCreationUpdate.Load()) + tk.MustExec("create user bbb") + handle.CheckFullData(t, false) + // trigger priv reload, but data for bbb is not really loaded + tk.MustExec("grant select on test.* to bbb") + priv = handle.Get() + // data for bbb is not loaded, because that user is not active + // So this is **counterintuitive**, but it's still the expected behavior. + require.False(t, priv.RequestVerification(nil, "bbb", "%", "test", "", "", mysql.SelectPriv)) + tk1 := testkit.NewTestKit(t, store) + // if user bbb login, everything works as expected + require.NoError(t, tk1.Session().Auth(&auth.UserIdentity{Username: "bbb", Hostname: "localhost"}, nil, nil, nil)) + priv = handle.Get() + require.True(t, priv.RequestVerification(nil, "bbb", "%", "test", "", "", mysql.SelectPriv)) + + // 3. change the variable and check again + tk.MustExec("set @@global.tidb_accelerate_user_creation_update = off") + tk.MustQuery("select @@global.tidb_accelerate_user_creation_update").Check(testkit.Rows("0")) + tk.MustExec("drop user aaa") + handle.CheckFullData(t, true) + priv = handle.Get() + require.True(t, priv.RequestVerification(nil, "bbb", "%", "test", "", "", mysql.SelectPriv)) +} diff --git a/pkg/privilege/privileges/tidb_auth_token_test.go b/pkg/privilege/privileges/tidb_auth_token_test.go index 791a6a943e1eb..a16d5c38fb959 100644 --- a/pkg/privilege/privileges/tidb_auth_token_test.go +++ b/pkg/privilege/privileges/tidb_auth_token_test.go @@ -486,4 +486,8 @@ func (p *MySQLPrivilege) RoleGraph() map[string]roleGraphEdgesTable { return p.roleGraph } +func (h *Handle) CheckFullData(t *testing.T, value bool) { + require.True(t, h.fullData.Load() == value) +} + var NewMySQLPrivilege = newMySQLPrivilege diff --git a/pkg/sessionctx/vardef/tidb_vars.go b/pkg/sessionctx/vardef/tidb_vars.go new file mode 100644 index 0000000000000..6efe98cb3ffd7 --- /dev/null +++ b/pkg/sessionctx/vardef/tidb_vars.go @@ -0,0 +1,2020 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package vardef + +import ( + "fmt" + "math" + "strconv" + "strings" + goatomic "sync/atomic" + "time" + + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/executor/join/joinversion" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/util/memory" + "github.com/pingcap/tidb/pkg/util/paging" + "github.com/pingcap/tidb/pkg/util/size" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/atomic" +) + +/* + Steps to add a new TiDB specific system variable: + + 1. Add a new variable name with comment in this file. + 2. Add the default value of the new variable in this file. + 3. Add SysVar instance in 'defaultSysVars' slice. +*/ + +// TiDB system variable names that only in session scope. +const ( + TiDBDDLSlowOprThreshold = "ddl_slow_threshold" + + // TiDBSnapshot is used for reading history data, the default value is empty string. + // The value can be a datetime string like '2017-11-11 20:20:20' or a tso string. When this variable is set, the session reads history data of that time. + TiDBSnapshot = "tidb_snapshot" + + // TiDBOptAggPushDown is used to enable/disable the optimizer rule of aggregation push down. + TiDBOptAggPushDown = "tidb_opt_agg_push_down" + + // TiDBOptDeriveTopN is used to enable/disable the optimizer rule of deriving topN. + TiDBOptDeriveTopN = "tidb_opt_derive_topn" + + // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode + TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" + + TiDBOptMPPOuterJoinFixedBuildSide = "tidb_opt_mpp_outer_join_fixed_build_side" + + // TiDBOptDistinctAggPushDown is used to decide whether agg with distinct should be pushed to tikv/tiflash. + TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" + + // TiDBOptSkewDistinctAgg is used to indicate the distinct agg has data skew + TiDBOptSkewDistinctAgg = "tidb_opt_skew_distinct_agg" + + // TiDBOpt3StageDistinctAgg is used to indicate whether to plan and execute the distinct agg in 3 stages + TiDBOpt3StageDistinctAgg = "tidb_opt_three_stage_distinct_agg" + + // TiDBOptEnable3StageMultiDistinctAgg is used to indicate whether to plan and execute the multi distinct agg in 3 stages + TiDBOptEnable3StageMultiDistinctAgg = "tidb_opt_enable_three_stage_multi_distinct_agg" + + TiDBOptExplainNoEvaledSubQuery = "tidb_opt_enable_non_eval_scalar_subquery" + + // TiDBBCJThresholdSize is used to limit the size of small table for mpp broadcast join. + // Its unit is bytes, if the size of small table is larger than it, we will not use bcj. + TiDBBCJThresholdSize = "tidb_broadcast_join_threshold_size" + + // TiDBBCJThresholdCount is used to limit the count of small table for mpp broadcast join. + // If we can't estimate the size of one side of join child, we will check if its row number exceeds this limitation. + TiDBBCJThresholdCount = "tidb_broadcast_join_threshold_count" + + // TiDBPreferBCJByExchangeDataSize indicates the method used to choose mpp broadcast join + TiDBPreferBCJByExchangeDataSize = "tidb_prefer_broadcast_join_by_exchange_data_size" + + // TiDBOptWriteRowID is used to enable/disable the operations of insert、replace and update to _tidb_rowid. + TiDBOptWriteRowID = "tidb_opt_write_row_id" + + // TiDBAutoAnalyzeRatio will run if (table modify count)/(table row count) is greater than this value. + TiDBAutoAnalyzeRatio = "tidb_auto_analyze_ratio" + + // TiDBAutoAnalyzeStartTime will run if current time is within start time and end time. + TiDBAutoAnalyzeStartTime = "tidb_auto_analyze_start_time" + TiDBAutoAnalyzeEndTime = "tidb_auto_analyze_end_time" + + // TiDBChecksumTableConcurrency is used to speed up the ADMIN CHECKSUM TABLE + // statement, when a table has multiple indices, those indices can be + // scanned concurrently, with the cost of higher system performance impact. + TiDBChecksumTableConcurrency = "tidb_checksum_table_concurrency" + + // TiDBCurrentTS is used to get the current transaction timestamp. + // It is read-only. + TiDBCurrentTS = "tidb_current_ts" + + // TiDBLastTxnInfo is used to get the last transaction info within the current session. + TiDBLastTxnInfo = "tidb_last_txn_info" + + // TiDBLastQueryInfo is used to get the last query info within the current session. + TiDBLastQueryInfo = "tidb_last_query_info" + + // TiDBLastDDLInfo is used to get the last ddl info within the current session. + TiDBLastDDLInfo = "tidb_last_ddl_info" + + // TiDBLastPlanReplayerToken is used to get the last plan replayer token within the current session + TiDBLastPlanReplayerToken = "tidb_last_plan_replayer_token" + + // TiDBConfig is a read-only variable that shows the config of the current server. + TiDBConfig = "tidb_config" + + // TiDBBatchInsert is used to enable/disable auto-split insert data. If set this option on, insert executor will automatically + // insert data into multiple batches and use a single txn for each batch. This will be helpful when inserting large data. + TiDBBatchInsert = "tidb_batch_insert" + + // TiDBBatchDelete is used to enable/disable auto-split delete data. If set this option on, delete executor will automatically + // split data into multiple batches and use a single txn for each batch. This will be helpful when deleting large data. + TiDBBatchDelete = "tidb_batch_delete" + + // TiDBBatchCommit is used to enable/disable auto-split the transaction. + // If set this option on, the transaction will be committed when it reaches stmt-count-limit and starts a new transaction. + TiDBBatchCommit = "tidb_batch_commit" + + // TiDBDMLBatchSize is used to split the insert/delete data into small batches. + // It only takes effort when tidb_batch_insert/tidb_batch_delete is on. + // Its default value is 20000. When the row size is large, 20k rows could be larger than 100MB. + // User could change it to a smaller one to avoid breaking the transaction size limitation. + TiDBDMLBatchSize = "tidb_dml_batch_size" + + // The following session variables controls the memory quota during query execution. + + // TiDBMemQuotaQuery controls the memory quota of a query. + TiDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. + // TiDBMemQuotaApplyCache controls the memory quota of a query. + TiDBMemQuotaApplyCache = "tidb_mem_quota_apply_cache" + + // TiDBGeneralLog is used to log every query in the server in info level. + TiDBGeneralLog = "tidb_general_log" + + // TiDBLogFileMaxDays is used to log every query in the server in info level. + TiDBLogFileMaxDays = "tidb_log_file_max_days" + + // TiDBPProfSQLCPU is used to add label sql label to pprof result. + TiDBPProfSQLCPU = "tidb_pprof_sql_cpu" + + // TiDBRetryLimit is the maximum number of retries when committing a transaction. + TiDBRetryLimit = "tidb_retry_limit" + + // TiDBDisableTxnAutoRetry disables transaction auto retry. + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBDisableTxnAutoRetry = "tidb_disable_txn_auto_retry" + + // TiDBEnableChunkRPC enables TiDB to use Chunk format for coprocessor requests. + TiDBEnableChunkRPC = "tidb_enable_chunk_rpc" + + // TiDBOptimizerSelectivityLevel is used to control the selectivity estimation level. + TiDBOptimizerSelectivityLevel = "tidb_optimizer_selectivity_level" + + // TiDBOptimizerEnableNewOnlyFullGroupByCheck is used to open the newly only_full_group_by check by maintaining functional dependency. + TiDBOptimizerEnableNewOnlyFullGroupByCheck = "tidb_enable_new_only_full_group_by_check" + + TiDBOptimizerEnableOuterJoinReorder = "tidb_enable_outer_join_reorder" + + // TiDBOptimizerEnableNAAJ is used to open the newly null-aware anti join + TiDBOptimizerEnableNAAJ = "tidb_enable_null_aware_anti_join" + + // TiDBTxnMode is used to control the transaction behavior. + TiDBTxnMode = "tidb_txn_mode" + + // TiDBRowFormatVersion is used to control tidb row format version current. + TiDBRowFormatVersion = "tidb_row_format_version" + + // TiDBEnableRowLevelChecksum is used to control whether to append checksum to row values. + TiDBEnableRowLevelChecksum = "tidb_enable_row_level_checksum" + + // TiDBEnableTablePartition is used to control table partition feature. + // The valid value include auto/on/off: + // on or auto: enable table partition if the partition type is implemented. + // off: always disable table partition. + TiDBEnableTablePartition = "tidb_enable_table_partition" + + // TiDBEnableListTablePartition is used to control list table partition feature. + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBEnableListTablePartition = "tidb_enable_list_partition" + + // TiDBSkipIsolationLevelCheck is used to control whether to return error when set unsupported transaction + // isolation level. + TiDBSkipIsolationLevelCheck = "tidb_skip_isolation_level_check" + + // TiDBLowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds + TiDBLowResolutionTSO = "tidb_low_resolution_tso" + + // TiDBReplicaRead is used for reading data from replicas, followers for example. + TiDBReplicaRead = "tidb_replica_read" + + // TiDBAdaptiveClosestReadThreshold is for reading data from closest replicas(with same 'zone' label). + // TiKV client should send read request to the closest replica(leader/follower) if the estimated response + // size exceeds this threshold; otherwise, this request should be sent to leader. + // This variable only take effect when `tidb_replica_read` is 'closest-adaptive'. + TiDBAdaptiveClosestReadThreshold = "tidb_adaptive_closest_read_threshold" + + // TiDBAllowRemoveAutoInc indicates whether a user can drop the auto_increment column attribute or not. + TiDBAllowRemoveAutoInc = "tidb_allow_remove_auto_inc" + + // TiDBMultiStatementMode enables multi statement at the risk of SQL injection + // provides backwards compatibility + TiDBMultiStatementMode = "tidb_multi_statement_mode" + + // TiDBEvolvePlanTaskMaxTime controls the max time of a single evolution task. + TiDBEvolvePlanTaskMaxTime = "tidb_evolve_plan_task_max_time" + + // TiDBEvolvePlanTaskStartTime is the start time of evolution task. + TiDBEvolvePlanTaskStartTime = "tidb_evolve_plan_task_start_time" + // TiDBEvolvePlanTaskEndTime is the end time of evolution task. + TiDBEvolvePlanTaskEndTime = "tidb_evolve_plan_task_end_time" + + // TiDBSlowLogThreshold is used to set the slow log threshold in the server. + TiDBSlowLogThreshold = "tidb_slow_log_threshold" + + // TiDBSlowTxnLogThreshold is used to set the slow transaction log threshold in the server. + TiDBSlowTxnLogThreshold = "tidb_slow_txn_log_threshold" + + // TiDBRecordPlanInSlowLog is used to log the plan of the slow query. + TiDBRecordPlanInSlowLog = "tidb_record_plan_in_slow_log" + + // TiDBEnableSlowLog enables TiDB to log slow queries. + TiDBEnableSlowLog = "tidb_enable_slow_log" + + // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. + TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" + + // TiDBFoundInPlanCache indicates whether the last statement was found in plan cache + TiDBFoundInPlanCache = "last_plan_from_cache" + + // TiDBFoundInBinding indicates whether the last statement was matched with the hints in the binding. + TiDBFoundInBinding = "last_plan_from_binding" + + // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. + TiDBAllowAutoRandExplicitInsert = "allow_auto_random_explicit_insert" + + // TiDBTxnReadTS indicates the next transaction should be staleness transaction and provide the startTS + TiDBTxnReadTS = "tx_read_ts" + + // TiDBReadStaleness indicates the staleness duration for following statement + TiDBReadStaleness = "tidb_read_staleness" + + // TiDBEnablePaging indicates whether paging is enabled in coprocessor requests. + TiDBEnablePaging = "tidb_enable_paging" + + // TiDBReadConsistency indicates whether the autocommit read statement goes through TiKV RC. + TiDBReadConsistency = "tidb_read_consistency" + + // TiDBSysdateIsNow is the name of the `tidb_sysdate_is_now` system variable + TiDBSysdateIsNow = "tidb_sysdate_is_now" + + // RequireSecureTransport indicates the secure mode for data transport + RequireSecureTransport = "require_secure_transport" + + // TiFlashFastScan indicates whether use fast scan in tiflash. + TiFlashFastScan = "tiflash_fastscan" + + // TiDBEnableUnsafeSubstitute indicates whether to enable generate column takes unsafe substitute. + TiDBEnableUnsafeSubstitute = "tidb_enable_unsafe_substitute" + + // TiDBEnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. + TiDBEnableTiFlashReadForWriteStmt = "tidb_enable_tiflash_read_for_write_stmt" + + // TiDBUseAlloc indicates whether the last statement used chunk alloc + TiDBUseAlloc = "last_sql_use_alloc" + + // TiDBExplicitRequestSourceType indicates the source of the request, it's a complement of RequestSourceType. + // The value maybe "lightning", "br", "dumpling" etc. + TiDBExplicitRequestSourceType = "tidb_request_source_type" +) + +// TiDB system variable names that both in session and global scope. +const ( + // TiDBBuildStatsConcurrency specifies the number of concurrent workers used for analyzing tables or partitions. + // When multiple tables or partitions are specified in the analyze statement, TiDB will process them concurrently. + // Additionally, this setting controls the concurrency for building NDV (Number of Distinct Values) for special indexes, + // such as generated columns composed indexes. + TiDBBuildStatsConcurrency = "tidb_build_stats_concurrency" + + // TiDBBuildSamplingStatsConcurrency is used to control the concurrency of building stats using sampling. + // 1. The number of concurrent workers to merge FMSketches and Sample Data from different regions. + // 2. The number of concurrent workers to build TopN and Histogram concurrently. + TiDBBuildSamplingStatsConcurrency = "tidb_build_sampling_stats_concurrency" + + // TiDBDistSQLScanConcurrency is used to set the concurrency of a distsql scan task. + // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. + // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. + // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. + TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" + + // TiDBAnalyzeDistSQLScanConcurrency is the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). + // For auto analyze, the value is controlled by tidb_sysproc_scan_concurrency variable. + TiDBAnalyzeDistSQLScanConcurrency = "tidb_analyze_distsql_scan_concurrency" + + // TiDBOptInSubqToJoinAndAgg is used to enable/disable the optimizer rule of rewriting IN subquery. + TiDBOptInSubqToJoinAndAgg = "tidb_opt_insubq_to_join_and_agg" + + // TiDBOptPreferRangeScan is used to enable/disable the optimizer to always prefer range scan over table scan, ignoring their costs. + TiDBOptPreferRangeScan = "tidb_opt_prefer_range_scan" + + // TiDBOptEnableCorrelationAdjustment is used to indicates if enable correlation adjustment. + TiDBOptEnableCorrelationAdjustment = "tidb_opt_enable_correlation_adjustment" + + // TiDBOptLimitPushDownThreshold determines if push Limit or TopN down to TiKV forcibly. + TiDBOptLimitPushDownThreshold = "tidb_opt_limit_push_down_threshold" + + // TiDBOptCorrelationThreshold is a guard to enable row count estimation using column order correlation. + TiDBOptCorrelationThreshold = "tidb_opt_correlation_threshold" + + // TiDBOptCorrelationExpFactor is an exponential factor to control heuristic approach when tidb_opt_correlation_threshold is not satisfied. + TiDBOptCorrelationExpFactor = "tidb_opt_correlation_exp_factor" + + // TiDBOptCPUFactor is the CPU cost of processing one expression for one row. + TiDBOptCPUFactor = "tidb_opt_cpu_factor" + // TiDBOptCopCPUFactor is the CPU cost of processing one expression for one row in coprocessor. + TiDBOptCopCPUFactor = "tidb_opt_copcpu_factor" + // TiDBOptTiFlashConcurrencyFactor is concurrency number of tiflash computation. + TiDBOptTiFlashConcurrencyFactor = "tidb_opt_tiflash_concurrency_factor" + // TiDBOptNetworkFactor is the network cost of transferring 1 byte data. + TiDBOptNetworkFactor = "tidb_opt_network_factor" + // TiDBOptScanFactor is the IO cost of scanning 1 byte data on TiKV. + TiDBOptScanFactor = "tidb_opt_scan_factor" + // TiDBOptDescScanFactor is the IO cost of scanning 1 byte data on TiKV in desc order. + TiDBOptDescScanFactor = "tidb_opt_desc_factor" + // TiDBOptSeekFactor is the IO cost of seeking the start value in a range on TiKV or TiFlash. + TiDBOptSeekFactor = "tidb_opt_seek_factor" + // TiDBOptMemoryFactor is the memory cost of storing one tuple. + TiDBOptMemoryFactor = "tidb_opt_memory_factor" + // TiDBOptDiskFactor is the IO cost of reading/writing one byte to temporary disk. + TiDBOptDiskFactor = "tidb_opt_disk_factor" + // TiDBOptConcurrencyFactor is the CPU cost of additional one goroutine. + TiDBOptConcurrencyFactor = "tidb_opt_concurrency_factor" + // TiDBOptForceInlineCTE is used to enable/disable inline CTE + TiDBOptForceInlineCTE = "tidb_opt_force_inline_cte" + + // TiDBIndexJoinBatchSize is used to set the batch size of an index lookup join. + // The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor. + // This value controls how much of data in a batch to do the index join. + // Large value may reduce the latency but consumes more system resource. + TiDBIndexJoinBatchSize = "tidb_index_join_batch_size" + + // TiDBIndexLookupSize is used for index lookup executor. + // The index lookup executor first scan a batch of handles from a index, then use those handles to lookup the table + // rows, this value controls how much of handles in a batch to do a lookup task. + // Small value sends more RPCs to TiKV, consume more system resource. + // Large value may do more work than needed if the query has a limit. + TiDBIndexLookupSize = "tidb_index_lookup_size" + + // TiDBIndexLookupConcurrency is used for index lookup executor. + // A lookup task may have 'tidb_index_lookup_size' of handles at maximum, the handles may be distributed + // in many TiKV nodes, we execute multiple concurrent index lookup tasks concurrently to reduce the time + // waiting for a task to finish. + // Set this value higher may reduce the latency but consumes more system resource. + // tidb_index_lookup_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBIndexLookupConcurrency = "tidb_index_lookup_concurrency" + + // TiDBIndexLookupJoinConcurrency is used for index lookup join executor. + // IndexLookUpJoin starts "tidb_index_lookup_join_concurrency" inner workers + // to fetch inner rows and join the matched (outer, inner) row pairs. + // tidb_index_lookup_join_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBIndexLookupJoinConcurrency = "tidb_index_lookup_join_concurrency" + + // TiDBIndexSerialScanConcurrency is used for controlling the concurrency of index scan operation + // when we need to keep the data output order the same as the order of index data. + TiDBIndexSerialScanConcurrency = "tidb_index_serial_scan_concurrency" + + // TiDBMaxChunkSize is used to control the max chunk size during query execution. + TiDBMaxChunkSize = "tidb_max_chunk_size" + + // TiDBAllowBatchCop means if we should send batch coprocessor to TiFlash. It can be set to 0, 1 and 2. + // 0 means never use batch cop, 1 means use batch cop in case of aggregation and join, 2, means to force sending batch cop for any query. + // The default value is 0 + TiDBAllowBatchCop = "tidb_allow_batch_cop" + + // TiDBShardRowIDBits means all the tables created in the current session will be sharded. + // The default value is 0 + TiDBShardRowIDBits = "tidb_shard_row_id_bits" + + // TiDBPreSplitRegions means all the tables created in the current session will be pre-splited. + // The default value is 0 + TiDBPreSplitRegions = "tidb_pre_split_regions" + + // TiDBAllowMPPExecution means if we should use mpp way to execute query or not. + // Default value is `true`, means to be determined by the optimizer. + // Value set to `false` means never use mpp. + TiDBAllowMPPExecution = "tidb_allow_mpp" + + // TiDBAllowTiFlashCop means we only use MPP mode to query data. + // Default value is `true`, means to be determined by the optimizer. + // Value set to `false` means we may fall back to TiFlash cop plan if possible. + TiDBAllowTiFlashCop = "tidb_allow_tiflash_cop" + + // TiDBHashExchangeWithNewCollation means if hash exchange is supported when new collation is on. + // Default value is `true`, means support hash exchange when new collation is on. + // Value set to `false` means not support hash exchange when new collation is on. + TiDBHashExchangeWithNewCollation = "tidb_hash_exchange_with_new_collation" + + // TiDBEnforceMPPExecution means if we should enforce mpp way to execute query or not. + // Default value is `false`, means to be determined by variable `tidb_allow_mpp`. + // Value set to `true` means enforce use mpp. + // Note if you want to set `tidb_enforce_mpp` to `true`, you must set `tidb_allow_mpp` to `true` first. + TiDBEnforceMPPExecution = "tidb_enforce_mpp" + + // TiDBMaxTiFlashThreads is the maximum number of threads to execute the request which is pushed down to tiflash. + // Default value is -1, means it will not be pushed down to tiflash. + // If the value is bigger than -1, it will be pushed down to tiflash and used to create db context in tiflash. + TiDBMaxTiFlashThreads = "tidb_max_tiflash_threads" + + // TiDBMaxBytesBeforeTiFlashExternalJoin is the maximum bytes used by a TiFlash join before spill to disk + TiDBMaxBytesBeforeTiFlashExternalJoin = "tidb_max_bytes_before_tiflash_external_join" + + // TiDBMaxBytesBeforeTiFlashExternalGroupBy is the maximum bytes used by a TiFlash hash aggregation before spill to disk + TiDBMaxBytesBeforeTiFlashExternalGroupBy = "tidb_max_bytes_before_tiflash_external_group_by" + + // TiDBMaxBytesBeforeTiFlashExternalSort is the maximum bytes used by a TiFlash sort/TopN before spill to disk + TiDBMaxBytesBeforeTiFlashExternalSort = "tidb_max_bytes_before_tiflash_external_sort" + + // TiFlashMemQuotaQueryPerNode is the maximum bytes used by a TiFlash Query on each TiFlash node + TiFlashMemQuotaQueryPerNode = "tiflash_mem_quota_query_per_node" + + // TiFlashQuerySpillRatio is the threshold that TiFlash will trigger auto spill when the memory usage is above this percentage + TiFlashQuerySpillRatio = "tiflash_query_spill_ratio" + + // TiDBMPPStoreFailTTL is the unavailable time when a store is detected failed. During that time, tidb will not send any task to + // TiFlash even though the failed TiFlash node has been recovered. + TiDBMPPStoreFailTTL = "tidb_mpp_store_fail_ttl" + + // TiDBInitChunkSize is used to control the init chunk size during query execution. + TiDBInitChunkSize = "tidb_init_chunk_size" + + // TiDBMinPagingSize is used to control the min paging size in the coprocessor paging protocol. + TiDBMinPagingSize = "tidb_min_paging_size" + + // TiDBMaxPagingSize is used to control the max paging size in the coprocessor paging protocol. + TiDBMaxPagingSize = "tidb_max_paging_size" + + // TiDBEnableCascadesPlanner is used to control whether to enable the cascades planner. + TiDBEnableCascadesPlanner = "tidb_enable_cascades_planner" + + // TiDBSkipUTF8Check skips the UTF8 validate process, validate UTF8 has performance cost, if we can make sure + // the input string values are valid, we can skip the check. + TiDBSkipUTF8Check = "tidb_skip_utf8_check" + + // TiDBSkipASCIICheck skips the ASCII validate process + // old tidb may already have fields with invalid ASCII bytes + // disable ASCII validate can guarantee a safe replication + TiDBSkipASCIICheck = "tidb_skip_ascii_check" + + // TiDBHashJoinConcurrency is used for hash join executor. + // The hash join outer executor starts multiple concurrent join workers to probe the hash table. + // tidb_hash_join_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBHashJoinConcurrency = "tidb_hash_join_concurrency" + + // TiDBProjectionConcurrency is used for projection operator. + // This variable controls the worker number of projection operator. + // tidb_projection_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBProjectionConcurrency = "tidb_projection_concurrency" + + // TiDBHashAggPartialConcurrency is used for hash agg executor. + // The hash agg executor starts multiple concurrent partial workers to do partial aggregate works. + // tidb_hashagg_partial_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBHashAggPartialConcurrency = "tidb_hashagg_partial_concurrency" + + // TiDBHashAggFinalConcurrency is used for hash agg executor. + // The hash agg executor starts multiple concurrent final workers to do final aggregate works. + // tidb_hashagg_final_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBHashAggFinalConcurrency = "tidb_hashagg_final_concurrency" + + // TiDBWindowConcurrency is used for window parallel executor. + // tidb_window_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBWindowConcurrency = "tidb_window_concurrency" + + // TiDBMergeJoinConcurrency is used for merge join parallel executor + TiDBMergeJoinConcurrency = "tidb_merge_join_concurrency" + + // TiDBStreamAggConcurrency is used for stream aggregation parallel executor. + // tidb_stream_agg_concurrency is deprecated, use tidb_executor_concurrency instead. + TiDBStreamAggConcurrency = "tidb_streamagg_concurrency" + + // TiDBIndexMergeIntersectionConcurrency is used for parallel worker of index merge intersection. + TiDBIndexMergeIntersectionConcurrency = "tidb_index_merge_intersection_concurrency" + + // TiDBEnableParallelApply is used for parallel apply. + TiDBEnableParallelApply = "tidb_enable_parallel_apply" + + // TiDBBackoffLockFast is used for tikv backoff base time in milliseconds. + TiDBBackoffLockFast = "tidb_backoff_lock_fast" + + // TiDBBackOffWeight is used to control the max back off time in TiDB. + // The default maximum back off time is a small value. + // BackOffWeight could multiply it to let the user adjust the maximum time for retrying. + // Only positive integers can be accepted, which means that the maximum back off time can only grow. + TiDBBackOffWeight = "tidb_backoff_weight" + + // TiDBDDLReorgWorkerCount defines the count of ddl reorg workers. + TiDBDDLReorgWorkerCount = "tidb_ddl_reorg_worker_cnt" + + // TiDBDDLFlashbackConcurrency defines the count of ddl flashback workers. + TiDBDDLFlashbackConcurrency = "tidb_ddl_flashback_concurrency" + + // TiDBDDLReorgBatchSize defines the transaction batch size of ddl reorg workers. + TiDBDDLReorgBatchSize = "tidb_ddl_reorg_batch_size" + + // TiDBDDLErrorCountLimit defines the count of ddl error limit. + TiDBDDLErrorCountLimit = "tidb_ddl_error_count_limit" + + // TiDBDDLReorgPriority defines the operations' priority of adding indices. + // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH + TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" + + // TiDBDDLReorgMaxWriteSpeed defines the max write limitation for the lightning local backend + TiDBDDLReorgMaxWriteSpeed = "tidb_ddl_reorg_max_write_speed" + + // TiDBEnableAutoIncrementInGenerated disables the mysql compatibility check on using auto-incremented columns in + // expression indexes and generated columns described here https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details. + TiDBEnableAutoIncrementInGenerated = "tidb_enable_auto_increment_in_generated" + + // TiDBPlacementMode is used to control the mode for placement + TiDBPlacementMode = "tidb_placement_mode" + + // TiDBMaxDeltaSchemaCount defines the max length of deltaSchemaInfos. + // deltaSchemaInfos is a queue that maintains the history of schema changes. + TiDBMaxDeltaSchemaCount = "tidb_max_delta_schema_count" + + // TiDBScatterRegion will scatter the regions for DDLs when it is "table" or "global", "" indicates not trigger scatter. + TiDBScatterRegion = "tidb_scatter_region" + + // TiDBWaitSplitRegionFinish defines the split region behaviour is sync or async. + TiDBWaitSplitRegionFinish = "tidb_wait_split_region_finish" + + // TiDBWaitSplitRegionTimeout uses to set the split and scatter region back off time. + TiDBWaitSplitRegionTimeout = "tidb_wait_split_region_timeout" + + // TiDBForcePriority defines the operations' priority of all statements. + // It can be "NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED" + TiDBForcePriority = "tidb_force_priority" + + // TiDBConstraintCheckInPlace indicates to check the constraint when the SQL executing. + // It could hurt the performance of bulking insert when it is ON. + TiDBConstraintCheckInPlace = "tidb_constraint_check_in_place" + + // TiDBEnableWindowFunction is used to control whether to enable the window function. + TiDBEnableWindowFunction = "tidb_enable_window_function" + + // TiDBEnablePipelinedWindowFunction is used to control whether to use pipelined window function, it only works when tidb_enable_window_function = true. + TiDBEnablePipelinedWindowFunction = "tidb_enable_pipelined_window_function" + + // TiDBEnableStrictDoubleTypeCheck is used to control table field double type syntax check. + TiDBEnableStrictDoubleTypeCheck = "tidb_enable_strict_double_type_check" + + // TiDBOptProjectionPushDown is used to control whether to pushdown projection to coprocessor. + TiDBOptProjectionPushDown = "tidb_opt_projection_push_down" + + // TiDBEnableVectorizedExpression is used to control whether to enable the vectorized expression evaluation. + TiDBEnableVectorizedExpression = "tidb_enable_vectorized_expression" + + // TiDBOptJoinReorderThreshold defines the threshold less than which + // we'll choose a rather time-consuming algorithm to calculate the join order. + TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" + + // TiDBSlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. + TiDBSlowQueryFile = "tidb_slow_query_file" + + // TiDBEnableFastAnalyze indicates to use fast analyze. + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBEnableFastAnalyze = "tidb_enable_fast_analyze" + + // TiDBExpensiveQueryTimeThreshold indicates the time threshold of expensive query. + TiDBExpensiveQueryTimeThreshold = "tidb_expensive_query_time_threshold" + + // TiDBExpensiveTxnTimeThreshold indicates the time threshold of expensive transaction. + TiDBExpensiveTxnTimeThreshold = "tidb_expensive_txn_time_threshold" + + // TiDBEnableIndexMerge indicates to generate IndexMergePath. + TiDBEnableIndexMerge = "tidb_enable_index_merge" + + // TiDBEnableNoopFuncs set true will enable using fake funcs(like get_lock release_lock) + TiDBEnableNoopFuncs = "tidb_enable_noop_functions" + + // TiDBEnableStmtSummary indicates whether the statement summary is enabled. + TiDBEnableStmtSummary = "tidb_enable_stmt_summary" + + // TiDBStmtSummaryInternalQuery indicates whether the statement summary contain internal query. + TiDBStmtSummaryInternalQuery = "tidb_stmt_summary_internal_query" + + // TiDBStmtSummaryRefreshInterval indicates the refresh interval in seconds for each statement summary. + TiDBStmtSummaryRefreshInterval = "tidb_stmt_summary_refresh_interval" + + // TiDBStmtSummaryHistorySize indicates the history size of each statement summary. + TiDBStmtSummaryHistorySize = "tidb_stmt_summary_history_size" + + // TiDBStmtSummaryMaxStmtCount indicates the max number of statements kept in memory. + TiDBStmtSummaryMaxStmtCount = "tidb_stmt_summary_max_stmt_count" + + // TiDBStmtSummaryMaxSQLLength indicates the max length of displayed normalized sql and sample sql. + TiDBStmtSummaryMaxSQLLength = "tidb_stmt_summary_max_sql_length" + + // TiDBIgnoreInlistPlanDigest enables TiDB to generate the same plan digest with SQL using different in-list arguments. + TiDBIgnoreInlistPlanDigest = "tidb_ignore_inlist_plan_digest" + + // TiDBCapturePlanBaseline indicates whether the capture of plan baselines is enabled. + TiDBCapturePlanBaseline = "tidb_capture_plan_baselines" + + // TiDBUsePlanBaselines indicates whether the use of plan baselines is enabled. + TiDBUsePlanBaselines = "tidb_use_plan_baselines" + + // TiDBEvolvePlanBaselines indicates whether the evolution of plan baselines is enabled. + TiDBEvolvePlanBaselines = "tidb_evolve_plan_baselines" + + // TiDBOptEnableFuzzyBinding indicates whether to enable the universal binding. + TiDBOptEnableFuzzyBinding = "tidb_opt_enable_fuzzy_binding" + + // TiDBEnableExtendedStats indicates whether the extended statistics feature is enabled. + TiDBEnableExtendedStats = "tidb_enable_extended_stats" + + // TiDBIsolationReadEngines indicates the tidb only read from the stores whose engine type is involved in IsolationReadEngines. + // Now, only support TiKV and TiFlash. + TiDBIsolationReadEngines = "tidb_isolation_read_engines" + + // TiDBStoreLimit indicates the limit of sending request to a store, 0 means without limit. + TiDBStoreLimit = "tidb_store_limit" + + // TiDBMetricSchemaStep indicates the step when query metric schema. + TiDBMetricSchemaStep = "tidb_metric_query_step" + + // TiDBCDCWriteSource indicates the following data is written by TiCDC if it is not 0. + TiDBCDCWriteSource = "tidb_cdc_write_source" + + // TiDBMetricSchemaRangeDuration indicates the range duration when query metric schema. + TiDBMetricSchemaRangeDuration = "tidb_metric_query_range_duration" + + // TiDBEnableCollectExecutionInfo indicates that whether execution info is collected. + TiDBEnableCollectExecutionInfo = "tidb_enable_collect_execution_info" + + // TiDBExecutorConcurrency is used for controlling the concurrency of all types of executors. + TiDBExecutorConcurrency = "tidb_executor_concurrency" + + // TiDBEnableClusteredIndex indicates if clustered index feature is enabled. + TiDBEnableClusteredIndex = "tidb_enable_clustered_index" + + // TiDBEnableGlobalIndex means if we could create an global index on a partition table or not. + // Deprecated, will always be ON + TiDBEnableGlobalIndex = "tidb_enable_global_index" + + // TiDBPartitionPruneMode indicates the partition prune mode used. + TiDBPartitionPruneMode = "tidb_partition_prune_mode" + + // TiDBRedactLog indicates that whether redact log. + TiDBRedactLog = "tidb_redact_log" + + // TiDBRestrictedReadOnly is meant for the cloud admin to toggle the cluster read only + TiDBRestrictedReadOnly = "tidb_restricted_read_only" + + // TiDBSuperReadOnly is tidb's variant of mysql's super_read_only, which has some differences from mysql's super_read_only. + TiDBSuperReadOnly = "tidb_super_read_only" + + // TiDBShardAllocateStep indicates the max size of continuous rowid shard in one transaction. + TiDBShardAllocateStep = "tidb_shard_allocate_step" + // TiDBEnableTelemetry indicates that whether usage data report to PingCAP is enabled. + // Deprecated: it is 'off' always since Telemetry has been removed from TiDB. + TiDBEnableTelemetry = "tidb_enable_telemetry" + + // TiDBMemoryUsageAlarmRatio indicates the alarm threshold when memory usage of the tidb-server exceeds. + TiDBMemoryUsageAlarmRatio = "tidb_memory_usage_alarm_ratio" + + // TiDBMemoryUsageAlarmKeepRecordNum indicates the number of saved alarm files. + TiDBMemoryUsageAlarmKeepRecordNum = "tidb_memory_usage_alarm_keep_record_num" + + // TiDBEnableRateLimitAction indicates whether enabled ratelimit action + TiDBEnableRateLimitAction = "tidb_enable_rate_limit_action" + + // TiDBEnableAsyncCommit indicates whether to enable the async commit feature. + TiDBEnableAsyncCommit = "tidb_enable_async_commit" + + // TiDBEnable1PC indicates whether to enable the one-phase commit feature. + TiDBEnable1PC = "tidb_enable_1pc" + + // TiDBGuaranteeLinearizability indicates whether to guarantee linearizability. + TiDBGuaranteeLinearizability = "tidb_guarantee_linearizability" + + // TiDBAnalyzeVersion indicates how tidb collects the analyzed statistics and how use to it. + TiDBAnalyzeVersion = "tidb_analyze_version" + + // TiDBAutoAnalyzePartitionBatchSize indicates the batch size for partition tables for auto analyze in dynamic mode + // Deprecated: This variable is deprecated, please do not use this variable. + TiDBAutoAnalyzePartitionBatchSize = "tidb_auto_analyze_partition_batch_size" + + // TiDBEnableIndexMergeJoin indicates whether to enable index merge join. + TiDBEnableIndexMergeJoin = "tidb_enable_index_merge_join" + + // TiDBTrackAggregateMemoryUsage indicates whether track the memory usage of aggregate function. + TiDBTrackAggregateMemoryUsage = "tidb_track_aggregate_memory_usage" + + // TiDBEnableExchangePartition indicates whether to enable exchange partition. + TiDBEnableExchangePartition = "tidb_enable_exchange_partition" + + // TiDBAllowFallbackToTiKV indicates the engine types whose unavailability triggers fallback to TiKV. + // Now we only support TiFlash. + TiDBAllowFallbackToTiKV = "tidb_allow_fallback_to_tikv" + + // TiDBEnableTopSQL indicates whether the top SQL is enabled. + TiDBEnableTopSQL = "tidb_enable_top_sql" + + // TiDBSourceID indicates the source ID of the TiDB server. + TiDBSourceID = "tidb_source_id" + + // TiDBTopSQLMaxTimeSeriesCount indicates the max number of statements been collected in each time series. + TiDBTopSQLMaxTimeSeriesCount = "tidb_top_sql_max_time_series_count" + + // TiDBTopSQLMaxMetaCount indicates the max capacity of the collect meta per second. + TiDBTopSQLMaxMetaCount = "tidb_top_sql_max_meta_count" + + // TiDBEnableLocalTxn indicates whether to enable Local Txn. + TiDBEnableLocalTxn = "tidb_enable_local_txn" + + // TiDBEnableMDL indicates whether to enable MDL. + TiDBEnableMDL = "tidb_enable_metadata_lock" + + // TiDBTSOClientBatchMaxWaitTime indicates the max value of the TSO Batch Wait interval time of PD client. + TiDBTSOClientBatchMaxWaitTime = "tidb_tso_client_batch_max_wait_time" + + // TiDBTxnCommitBatchSize is used to control the batch size of transaction commit related requests sent by TiDB to TiKV. + // If a single transaction has a large amount of writes, you can increase the batch size to improve the batch effect, + // setting too large will exceed TiKV's raft-entry-max-size limit and cause commit failure. + TiDBTxnCommitBatchSize = "tidb_txn_commit_batch_size" + + // TiDBEnableTSOFollowerProxy indicates whether to enable the TSO Follower Proxy feature of PD client. + TiDBEnableTSOFollowerProxy = "tidb_enable_tso_follower_proxy" + + // PDEnableFollowerHandleRegion indicates whether to enable the PD Follower handle region API. + PDEnableFollowerHandleRegion = "pd_enable_follower_handle_region" + + // TiDBEnableOrderedResultMode indicates if stabilize query results. + TiDBEnableOrderedResultMode = "tidb_enable_ordered_result_mode" + + // TiDBRemoveOrderbyInSubquery indicates whether to remove ORDER BY in subquery. + TiDBRemoveOrderbyInSubquery = "tidb_remove_orderby_in_subquery" + + // TiDBEnablePseudoForOutdatedStats indicates whether use pseudo for outdated stats + TiDBEnablePseudoForOutdatedStats = "tidb_enable_pseudo_for_outdated_stats" + + // TiDBRegardNULLAsPoint indicates whether regard NULL as point when optimizing + TiDBRegardNULLAsPoint = "tidb_regard_null_as_point" + + // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. + TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" + + // TiDBEnableLegacyInstanceScope indicates if instance scope can be set with SET SESSION. + TiDBEnableLegacyInstanceScope = "tidb_enable_legacy_instance_scope" + + // TiDBTableCacheLease indicates the read lock lease of a cached table. + TiDBTableCacheLease = "tidb_table_cache_lease" + + // TiDBStatsLoadSyncWait indicates the time sql execution will sync-wait for stats load. + TiDBStatsLoadSyncWait = "tidb_stats_load_sync_wait" + + // TiDBEnableMutationChecker indicates whether to check data consistency for mutations + TiDBEnableMutationChecker = "tidb_enable_mutation_checker" + // TiDBTxnAssertionLevel indicates how strict the assertion will be, which helps to detect and preventing data & + // index inconsistency problems. + TiDBTxnAssertionLevel = "tidb_txn_assertion_level" + + // TiDBIgnorePreparedCacheCloseStmt indicates whether to ignore close-stmt commands for prepared statements. + TiDBIgnorePreparedCacheCloseStmt = "tidb_ignore_prepared_cache_close_stmt" + + // TiDBEnableNewCostInterface is a internal switch to indicates whether to use the new cost calculation interface. + TiDBEnableNewCostInterface = "tidb_enable_new_cost_interface" + + // TiDBCostModelVersion is a internal switch to indicates the cost model version. + TiDBCostModelVersion = "tidb_cost_model_version" + + // TiDBIndexJoinDoubleReadPenaltyCostRate indicates whether to add some penalty cost to IndexJoin and how much of it. + // IndexJoin can cause plenty of extra double read tasks, which consume lots of resources and take a long time. + // Since the number of double read tasks is hard to estimated accurately, we leave this variable to let us can adjust this + // part of cost manually. + TiDBIndexJoinDoubleReadPenaltyCostRate = "tidb_index_join_double_read_penalty_cost_rate" + + // TiDBBatchPendingTiFlashCount indicates the maximum count of non-available TiFlash tables. + TiDBBatchPendingTiFlashCount = "tidb_batch_pending_tiflash_count" + + // TiDBQueryLogMaxLen is used to set the max length of the query in the log. + TiDBQueryLogMaxLen = "tidb_query_log_max_len" + + // TiDBEnableNoopVariables is used to indicate if noops appear in SHOW [GLOBAL] VARIABLES + TiDBEnableNoopVariables = "tidb_enable_noop_variables" + + // TiDBNonTransactionalIgnoreError is used to ignore error in non-transactional DMLs. + // When set to false, a non-transactional DML returns when it meets the first error. + // When set to true, a non-transactional DML finishes all batches even if errors are met in some batches. + TiDBNonTransactionalIgnoreError = "tidb_nontransactional_ignore_error" + + // Fine grained shuffle is disabled when TiFlashFineGrainedShuffleStreamCount is zero. + TiFlashFineGrainedShuffleStreamCount = "tiflash_fine_grained_shuffle_stream_count" + TiFlashFineGrainedShuffleBatchSize = "tiflash_fine_grained_shuffle_batch_size" + + // TiDBSimplifiedMetrics controls whether to unregister some unused metrics. + TiDBSimplifiedMetrics = "tidb_simplified_metrics" + + // TiDBMemoryDebugModeMinHeapInUse is used to set tidb memory debug mode trigger threshold. + // When set to 0, the function is disabled. + // When set to a negative integer, use memory debug mode to detect the issue of frequent allocation and release of memory. + // We do not actively trigger gc, and check whether the `tracker memory * (1+bias ratio) > heap in use` each 5s. + // When set to a positive integer, use memory debug mode to detect the issue of memory tracking inaccurate. + // We trigger runtime.GC() each 5s, and check whether the `tracker memory * (1+bias ratio) > heap in use`. + TiDBMemoryDebugModeMinHeapInUse = "tidb_memory_debug_mode_min_heap_inuse" + // TiDBMemoryDebugModeAlarmRatio is used set tidb memory debug mode bias ratio. Treat memory bias less than this ratio as noise. + TiDBMemoryDebugModeAlarmRatio = "tidb_memory_debug_mode_alarm_ratio" + + // TiDBEnableAnalyzeSnapshot indicates whether to read data on snapshot when collecting statistics. + // When set to false, ANALYZE reads the latest data. + // When set to true, ANALYZE reads data on the snapshot at the beginning of ANALYZE. + TiDBEnableAnalyzeSnapshot = "tidb_enable_analyze_snapshot" + + // TiDBDefaultStrMatchSelectivity controls some special cardinality estimation strategy for string match functions (like and regexp). + // When set to 0, Selectivity() will try to evaluate those functions with TopN and NULL in the stats to estimate, + // and the default selectivity and the selectivity for the histogram part will be 0.1. + // When set to (0, 1], Selectivity() will use the value of this variable as the default selectivity of those + // functions instead of the selectionFactor (0.8). + TiDBDefaultStrMatchSelectivity = "tidb_default_string_match_selectivity" + + // TiDBEnablePrepPlanCache indicates whether to enable prepared plan cache + TiDBEnablePrepPlanCache = "tidb_enable_prepared_plan_cache" + // TiDBPrepPlanCacheSize indicates the number of cached statements. + // This variable is deprecated, use tidb_session_plan_cache_size instead. + TiDBPrepPlanCacheSize = "tidb_prepared_plan_cache_size" + // TiDBEnablePrepPlanCacheMemoryMonitor indicates whether to enable prepared plan cache monitor + TiDBEnablePrepPlanCacheMemoryMonitor = "tidb_enable_prepared_plan_cache_memory_monitor" + + // TiDBEnableNonPreparedPlanCache indicates whether to enable non-prepared plan cache. + TiDBEnableNonPreparedPlanCache = "tidb_enable_non_prepared_plan_cache" + // TiDBEnableNonPreparedPlanCacheForDML indicates whether to enable non-prepared plan cache for DML statements. + TiDBEnableNonPreparedPlanCacheForDML = "tidb_enable_non_prepared_plan_cache_for_dml" + // TiDBNonPreparedPlanCacheSize controls the size of non-prepared plan cache. + // This variable is deprecated, use tidb_session_plan_cache_size instead. + TiDBNonPreparedPlanCacheSize = "tidb_non_prepared_plan_cache_size" + // TiDBPlanCacheMaxPlanSize controls the maximum size of a plan that can be cached. + TiDBPlanCacheMaxPlanSize = "tidb_plan_cache_max_plan_size" + // TiDBPlanCacheInvalidationOnFreshStats controls if plan cache will be invalidated automatically when + // related stats are analyzed after the plan cache is generated. + TiDBPlanCacheInvalidationOnFreshStats = "tidb_plan_cache_invalidation_on_fresh_stats" + // TiDBSessionPlanCacheSize controls the size of session plan cache. + TiDBSessionPlanCacheSize = "tidb_session_plan_cache_size" + + // TiDBEnableInstancePlanCache indicates whether to enable instance plan cache. + // If this variable is false, session-level plan cache will be used. + TiDBEnableInstancePlanCache = "tidb_enable_instance_plan_cache" + // TiDBInstancePlanCacheReservedPercentage indicates the percentage memory to evict. + TiDBInstancePlanCacheReservedPercentage = "tidb_instance_plan_cache_reserved_percentage" + // TiDBInstancePlanCacheMaxMemSize indicates the maximum memory size of instance plan cache. + TiDBInstancePlanCacheMaxMemSize = "tidb_instance_plan_cache_max_size" + + // TiDBConstraintCheckInPlacePessimistic controls whether to skip certain kinds of pessimistic locks. + TiDBConstraintCheckInPlacePessimistic = "tidb_constraint_check_in_place_pessimistic" + + // TiDBEnableForeignKey indicates whether to enable foreign key feature. + // TODO(crazycs520): remove this after foreign key GA. + TiDBEnableForeignKey = "tidb_enable_foreign_key" + + // TiDBOptRangeMaxSize is the max memory limit for ranges. When the optimizer estimates that the memory usage of complete + // ranges would exceed the limit, it chooses less accurate ranges such as full range. 0 indicates that there is no memory + // limit for ranges. + TiDBOptRangeMaxSize = "tidb_opt_range_max_size" + + // TiDBOptAdvancedJoinHint indicates whether the join method hint is compatible with join order hint. + TiDBOptAdvancedJoinHint = "tidb_opt_advanced_join_hint" + // TiDBOptUseInvisibleIndexes indicates whether to use invisible indexes. + TiDBOptUseInvisibleIndexes = "tidb_opt_use_invisible_indexes" + // TiDBAnalyzePartitionConcurrency is the number of concurrent workers to save statistics to the system tables. + TiDBAnalyzePartitionConcurrency = "tidb_analyze_partition_concurrency" + // TiDBMergePartitionStatsConcurrency indicates the concurrency when merge partition stats into global stats + TiDBMergePartitionStatsConcurrency = "tidb_merge_partition_stats_concurrency" + // TiDBEnableAsyncMergeGlobalStats indicates whether to enable async merge global stats + TiDBEnableAsyncMergeGlobalStats = "tidb_enable_async_merge_global_stats" + // TiDBOptPrefixIndexSingleScan indicates whether to do some optimizations to avoid double scan for prefix index. + // When set to true, `col is (not) null`(`col` is index prefix column) is regarded as index filter rather than table filter. + TiDBOptPrefixIndexSingleScan = "tidb_opt_prefix_index_single_scan" + + // TiDBEnableExternalTSRead indicates whether to enable read through an external ts + TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" + + // TiDBEnablePlanReplayerCapture indicates whether to enable plan replayer capture + TiDBEnablePlanReplayerCapture = "tidb_enable_plan_replayer_capture" + + // TiDBEnablePlanReplayerContinuousCapture indicates whether to enable continuous capture + TiDBEnablePlanReplayerContinuousCapture = "tidb_enable_plan_replayer_continuous_capture" + // TiDBEnableReusechunk indicates whether to enable chunk alloc + TiDBEnableReusechunk = "tidb_enable_reuse_chunk" + + // TiDBStoreBatchSize indicates the batch size of coprocessor in the same store. + TiDBStoreBatchSize = "tidb_store_batch_size" + + // MppExchangeCompressionMode indicates the data compression method in mpp exchange operator + MppExchangeCompressionMode = "mpp_exchange_compression_mode" + + // MppVersion indicates the mpp-version used to build mpp plan + MppVersion = "mpp_version" + + // TiDBPessimisticTransactionFairLocking controls whether fair locking for pessimistic transaction + // is enabled. + TiDBPessimisticTransactionFairLocking = "tidb_pessimistic_txn_fair_locking" + + // TiDBEnablePlanCacheForParamLimit controls whether prepare statement with parameterized limit can be cached + TiDBEnablePlanCacheForParamLimit = "tidb_enable_plan_cache_for_param_limit" + + // TiDBEnableINLJoinInnerMultiPattern indicates whether enable multi pattern for inner side of inl join + TiDBEnableINLJoinInnerMultiPattern = "tidb_enable_inl_join_inner_multi_pattern" + + // TiFlashComputeDispatchPolicy indicates how to dispatch task to tiflash_compute nodes. + TiFlashComputeDispatchPolicy = "tiflash_compute_dispatch_policy" + + // TiDBEnablePlanCacheForSubquery controls whether prepare statement with subquery can be cached + TiDBEnablePlanCacheForSubquery = "tidb_enable_plan_cache_for_subquery" + + // TiDBOptEnableLateMaterialization indicates whether to enable late materialization + TiDBOptEnableLateMaterialization = "tidb_opt_enable_late_materialization" + // TiDBLoadBasedReplicaReadThreshold is the wait duration threshold to enable replica read automatically. + TiDBLoadBasedReplicaReadThreshold = "tidb_load_based_replica_read_threshold" + + // TiDBOptOrderingIdxSelThresh is the threshold for optimizer to consider the ordering index. + TiDBOptOrderingIdxSelThresh = "tidb_opt_ordering_index_selectivity_threshold" + + // TiDBOptOrderingIdxSelRatio is the ratio the optimizer will assume applies when non indexed filtering rows are found + // via the ordering index. + TiDBOptOrderingIdxSelRatio = "tidb_opt_ordering_index_selectivity_ratio" + + // TiDBOptEnableMPPSharedCTEExecution indicates whether the optimizer try to build shared CTE scan during MPP execution. + TiDBOptEnableMPPSharedCTEExecution = "tidb_opt_enable_mpp_shared_cte_execution" + // TiDBOptFixControl makes the user able to control some details of the optimizer behavior. + TiDBOptFixControl = "tidb_opt_fix_control" + + // TiFlashReplicaRead is used to set the policy of TiFlash replica read when the query needs the TiFlash engine. + TiFlashReplicaRead = "tiflash_replica_read" + + // TiDBLockUnchangedKeys indicates whether to lock duplicate keys in INSERT IGNORE and REPLACE statements, + // or unchanged unique keys in UPDATE statements, see PR #42210 and #42713 + TiDBLockUnchangedKeys = "tidb_lock_unchanged_keys" + + // TiDBFastCheckTable enables fast check table. + TiDBFastCheckTable = "tidb_enable_fast_table_check" + + // TiDBAnalyzeSkipColumnTypes indicates the column types whose statistics would not be collected when executing the ANALYZE command. + TiDBAnalyzeSkipColumnTypes = "tidb_analyze_skip_column_types" + + // TiDBEnableCheckConstraint indicates whether to enable check constraint feature. + TiDBEnableCheckConstraint = "tidb_enable_check_constraint" + + // TiDBOptEnableHashJoin indicates whether to enable hash join. + TiDBOptEnableHashJoin = "tidb_opt_enable_hash_join" + + // TiDBHashJoinVersion indicates whether to use hash join implementation v2. + TiDBHashJoinVersion = "tidb_hash_join_version" + + // TiDBOptObjective indicates whether the optimizer should be more stable, predictable or more aggressive. + // Please see comments of SessionVars.OptObjective for details. + TiDBOptObjective = "tidb_opt_objective" + + // TiDBEnableParallelHashaggSpill is the name of the `tidb_enable_parallel_hashagg_spill` system variable + TiDBEnableParallelHashaggSpill = "tidb_enable_parallel_hashagg_spill" + + // TiDBTxnEntrySizeLimit indicates the max size of a entry in membuf. + TiDBTxnEntrySizeLimit = "tidb_txn_entry_size_limit" + + // TiDBSchemaCacheSize indicates the size of infoschema meta data which are cached in V2 implementation. + TiDBSchemaCacheSize = "tidb_schema_cache_size" + + // DivPrecisionIncrement indicates the number of digits by which to increase the scale of the result of + // division operations performed with the / operator. + DivPrecisionIncrement = "div_precision_increment" + + // TiDBEnableSharedLockPromotion indicates whether the `select for share` statement would be executed + // as `select for update` statements which do acquire pessimistic locks. + TiDBEnableSharedLockPromotion = "tidb_enable_shared_lock_promotion" + + // TiDBAccelerateUserCreationUpdate decides whether tidb will load & update the whole user's data in-memory. + TiDBAccelerateUserCreationUpdate = "tidb_accelerate_user_creation_update" +) + +// TiDB vars that have only global scope +const ( + // TiDBGCEnable turns garbage collection on or OFF + TiDBGCEnable = "tidb_gc_enable" + // TiDBGCRunInterval sets the interval that GC runs + TiDBGCRunInterval = "tidb_gc_run_interval" + // TiDBGCLifetime sets the retention window of older versions + TiDBGCLifetime = "tidb_gc_life_time" + // TiDBGCConcurrency sets the concurrency of garbage collection. -1 = AUTO value + TiDBGCConcurrency = "tidb_gc_concurrency" + // TiDBGCScanLockMode enables the green GC feature (deprecated) + TiDBGCScanLockMode = "tidb_gc_scan_lock_mode" + // TiDBGCMaxWaitTime sets max time for gc advances the safepoint delayed by active transactions + TiDBGCMaxWaitTime = "tidb_gc_max_wait_time" + // TiDBEnableEnhancedSecurity restricts SUPER users from certain operations. + TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" + // TiDBEnableHistoricalStats enables the historical statistics feature (default off) + TiDBEnableHistoricalStats = "tidb_enable_historical_stats" + // TiDBPersistAnalyzeOptions persists analyze options for later analyze and auto-analyze + TiDBPersistAnalyzeOptions = "tidb_persist_analyze_options" + // TiDBEnableColumnTracking enables collecting predicate columns. + // DEPRECATED: This variable is deprecated, please do not use this variable. + TiDBEnableColumnTracking = "tidb_enable_column_tracking" + // TiDBAnalyzeColumnOptions specifies the default column selection strategy for both manual and automatic analyze operations. + // It accepts two values: + // `PREDICATE`: Analyze only the columns that are used in the predicates of the query. + // `ALL`: Analyze all columns in the table. + TiDBAnalyzeColumnOptions = "tidb_analyze_column_options" + // TiDBDisableColumnTrackingTime records the last time TiDBEnableColumnTracking is set off. + // It is used to invalidate the collected predicate columns after turning off TiDBEnableColumnTracking, which avoids physical deletion. + // It doesn't have cache in memory, and we directly get/set the variable value from/to mysql.tidb. + // DEPRECATED: This variable is deprecated, please do not use this variable. + TiDBDisableColumnTrackingTime = "tidb_disable_column_tracking_time" + // TiDBStatsLoadPseudoTimeout indicates whether to fallback to pseudo stats after load timeout. + TiDBStatsLoadPseudoTimeout = "tidb_stats_load_pseudo_timeout" + // TiDBMemQuotaBindingCache indicates the memory quota for the bind cache. + TiDBMemQuotaBindingCache = "tidb_mem_quota_binding_cache" + // TiDBRCReadCheckTS indicates the tso optimization for read-consistency read is enabled. + TiDBRCReadCheckTS = "tidb_rc_read_check_ts" + // TiDBRCWriteCheckTs indicates whether some special write statements don't get latest tso from PD at RC + TiDBRCWriteCheckTs = "tidb_rc_write_check_ts" + // TiDBCommitterConcurrency controls the number of running concurrent requests in the commit phase. + TiDBCommitterConcurrency = "tidb_committer_concurrency" + // TiDBEnableBatchDML enables batch dml. + TiDBEnableBatchDML = "tidb_enable_batch_dml" + // TiDBStatsCacheMemQuota records stats cache quota + TiDBStatsCacheMemQuota = "tidb_stats_cache_mem_quota" + // TiDBMemQuotaAnalyze indicates the memory quota for all analyze jobs. + TiDBMemQuotaAnalyze = "tidb_mem_quota_analyze" + // TiDBEnableAutoAnalyze determines whether TiDB executes automatic analysis. + // In test, we disable it by default. See GlobalSystemVariableInitialValue for details. + TiDBEnableAutoAnalyze = "tidb_enable_auto_analyze" + // TiDBEnableAutoAnalyzePriorityQueue determines whether TiDB executes automatic analysis with priority queue. + TiDBEnableAutoAnalyzePriorityQueue = "tidb_enable_auto_analyze_priority_queue" + // TiDBMemOOMAction indicates what operation TiDB perform when a single SQL statement exceeds + // the memory quota specified by tidb_mem_quota_query and cannot be spilled to disk. + TiDBMemOOMAction = "tidb_mem_oom_action" + // TiDBPrepPlanCacheMemoryGuardRatio is used to prevent [performance.max-memory] from being exceeded + TiDBPrepPlanCacheMemoryGuardRatio = "tidb_prepared_plan_cache_memory_guard_ratio" + // TiDBMaxAutoAnalyzeTime is the max time that auto analyze can run. If auto analyze runs longer than the value, it + // will be killed. 0 indicates that there is no time limit. + TiDBMaxAutoAnalyzeTime = "tidb_max_auto_analyze_time" + // TiDBAutoAnalyzeConcurrency is the concurrency of the auto analyze + TiDBAutoAnalyzeConcurrency = "tidb_auto_analyze_concurrency" + // TiDBEnableDistTask indicates whether to enable the distributed execute background tasks(For example DDL, Import etc). + TiDBEnableDistTask = "tidb_enable_dist_task" + // TiDBEnableFastCreateTable indicates whether to enable the fast create table feature. + TiDBEnableFastCreateTable = "tidb_enable_fast_create_table" + // TiDBGenerateBinaryPlan indicates whether binary plan should be generated in slow log and statements summary. + TiDBGenerateBinaryPlan = "tidb_generate_binary_plan" + // TiDBEnableGCAwareMemoryTrack indicates whether to turn-on GC-aware memory track. + TiDBEnableGCAwareMemoryTrack = "tidb_enable_gc_aware_memory_track" + // TiDBEnableTmpStorageOnOOM controls whether to enable the temporary storage for some operators + // when a single SQL statement exceeds the memory quota specified by the memory quota. + TiDBEnableTmpStorageOnOOM = "tidb_enable_tmp_storage_on_oom" + // TiDBDDLEnableFastReorg indicates whether to use lighting backfill process for adding index. + TiDBDDLEnableFastReorg = "tidb_ddl_enable_fast_reorg" + // TiDBDDLDiskQuota used to set disk quota for lightning add index. + TiDBDDLDiskQuota = "tidb_ddl_disk_quota" + // TiDBCloudStorageURI used to set a cloud storage uri for ddl add index and import into. + TiDBCloudStorageURI = "tidb_cloud_storage_uri" + // TiDBAutoBuildStatsConcurrency is the number of concurrent workers to automatically analyze tables or partitions. + // It is very similar to the `tidb_build_stats_concurrency` variable, but it is used for the auto analyze feature. + TiDBAutoBuildStatsConcurrency = "tidb_auto_build_stats_concurrency" + // TiDBSysProcScanConcurrency is used to set the scan concurrency of for backend system processes, like auto-analyze. + // For now, it controls the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). + TiDBSysProcScanConcurrency = "tidb_sysproc_scan_concurrency" + // TiDBServerMemoryLimit indicates the memory limit of the tidb-server instance. + TiDBServerMemoryLimit = "tidb_server_memory_limit" + // TiDBServerMemoryLimitSessMinSize indicates the minimal memory used of a session, that becomes a candidate for session kill. + TiDBServerMemoryLimitSessMinSize = "tidb_server_memory_limit_sess_min_size" + // TiDBServerMemoryLimitGCTrigger indicates the gc percentage of the TiDBServerMemoryLimit. + TiDBServerMemoryLimitGCTrigger = "tidb_server_memory_limit_gc_trigger" + // TiDBEnableGOGCTuner is to enable GOGC tuner. it can tuner GOGC + TiDBEnableGOGCTuner = "tidb_enable_gogc_tuner" + // TiDBGOGCTunerThreshold is to control the threshold of GOGC tuner. + TiDBGOGCTunerThreshold = "tidb_gogc_tuner_threshold" + // TiDBGOGCTunerMaxValue is the max value of GOGC that GOGC tuner can change to. + TiDBGOGCTunerMaxValue = "tidb_gogc_tuner_max_value" + // TiDBGOGCTunerMinValue is the min value of GOGC that GOGC tuner can change to. + TiDBGOGCTunerMinValue = "tidb_gogc_tuner_min_value" + // TiDBExternalTS is the ts to read through when the `TiDBEnableExternalTsRead` is on + TiDBExternalTS = "tidb_external_ts" + // TiDBTTLJobEnable is used to enable/disable scheduling ttl job + TiDBTTLJobEnable = "tidb_ttl_job_enable" + // TiDBTTLScanBatchSize is used to control the batch size in the SELECT statement for TTL jobs + TiDBTTLScanBatchSize = "tidb_ttl_scan_batch_size" + // TiDBTTLDeleteBatchSize is used to control the batch size in the DELETE statement for TTL jobs + TiDBTTLDeleteBatchSize = "tidb_ttl_delete_batch_size" + // TiDBTTLDeleteRateLimit is used to control the delete rate limit for TTL jobs in each node + TiDBTTLDeleteRateLimit = "tidb_ttl_delete_rate_limit" + // TiDBTTLJobScheduleWindowStartTime is used to restrict the start time of the time window of scheduling the ttl jobs. + TiDBTTLJobScheduleWindowStartTime = "tidb_ttl_job_schedule_window_start_time" + // TiDBTTLJobScheduleWindowEndTime is used to restrict the end time of the time window of scheduling the ttl jobs. + TiDBTTLJobScheduleWindowEndTime = "tidb_ttl_job_schedule_window_end_time" + // TiDBTTLScanWorkerCount indicates the count of the scan workers in each TiDB node + TiDBTTLScanWorkerCount = "tidb_ttl_scan_worker_count" + // TiDBTTLDeleteWorkerCount indicates the count of the delete workers in each TiDB node + TiDBTTLDeleteWorkerCount = "tidb_ttl_delete_worker_count" + // PasswordReuseHistory limit a few passwords to reuse. + PasswordReuseHistory = "password_history" + // PasswordReuseTime limit how long passwords can be reused. + PasswordReuseTime = "password_reuse_interval" + // TiDBHistoricalStatsDuration indicates the duration to remain tidb historical stats + TiDBHistoricalStatsDuration = "tidb_historical_stats_duration" + // TiDBEnableHistoricalStatsForCapture indicates whether use historical stats in plan replayer capture + TiDBEnableHistoricalStatsForCapture = "tidb_enable_historical_stats_for_capture" + // TiDBEnableResourceControl indicates whether resource control feature is enabled + TiDBEnableResourceControl = "tidb_enable_resource_control" + // TiDBResourceControlStrictMode indicates whether resource control strict mode is enabled. + // When strict mode is enabled, user need certain privilege to change session or statement resource group. + TiDBResourceControlStrictMode = "tidb_resource_control_strict_mode" + // TiDBStmtSummaryEnablePersistent indicates whether to enable file persistence for stmtsummary. + TiDBStmtSummaryEnablePersistent = "tidb_stmt_summary_enable_persistent" + // TiDBStmtSummaryFilename indicates the file name written by stmtsummary. + TiDBStmtSummaryFilename = "tidb_stmt_summary_filename" + // TiDBStmtSummaryFileMaxDays indicates how many days the files written by stmtsummary will be kept. + TiDBStmtSummaryFileMaxDays = "tidb_stmt_summary_file_max_days" + // TiDBStmtSummaryFileMaxSize indicates the maximum size (in mb) of a single file written by stmtsummary. + TiDBStmtSummaryFileMaxSize = "tidb_stmt_summary_file_max_size" + // TiDBStmtSummaryFileMaxBackups indicates the maximum number of files written by stmtsummary. + TiDBStmtSummaryFileMaxBackups = "tidb_stmt_summary_file_max_backups" + // TiDBTTLRunningTasks limits the count of running ttl tasks. Default to 0, means 3 times the count of TiKV (or no + // limitation, if the storage is not TiKV). + TiDBTTLRunningTasks = "tidb_ttl_running_tasks" + // AuthenticationLDAPSASLAuthMethodName defines the authentication method used by LDAP SASL authentication plugin + AuthenticationLDAPSASLAuthMethodName = "authentication_ldap_sasl_auth_method_name" + // AuthenticationLDAPSASLCAPath defines the ca certificate to verify LDAP connection in LDAP SASL authentication plugin + AuthenticationLDAPSASLCAPath = "authentication_ldap_sasl_ca_path" + // AuthenticationLDAPSASLTLS defines whether to use TLS connection in LDAP SASL authentication plugin + AuthenticationLDAPSASLTLS = "authentication_ldap_sasl_tls" + // AuthenticationLDAPSASLServerHost defines the server host of LDAP server for LDAP SASL authentication plugin + AuthenticationLDAPSASLServerHost = "authentication_ldap_sasl_server_host" + // AuthenticationLDAPSASLServerPort defines the port of LDAP server for LDAP SASL authentication plugin + AuthenticationLDAPSASLServerPort = "authentication_ldap_sasl_server_port" + // AuthenticationLDAPSASLReferral defines whether to enable LDAP referral for LDAP SASL authentication plugin + AuthenticationLDAPSASLReferral = "authentication_ldap_sasl_referral" + // AuthenticationLDAPSASLUserSearchAttr defines the attribute of username in LDAP server + AuthenticationLDAPSASLUserSearchAttr = "authentication_ldap_sasl_user_search_attr" + // AuthenticationLDAPSASLBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. + AuthenticationLDAPSASLBindBaseDN = "authentication_ldap_sasl_bind_base_dn" + // AuthenticationLDAPSASLBindRootDN defines the `dn` of the user to login the LDAP server and perform search. + AuthenticationLDAPSASLBindRootDN = "authentication_ldap_sasl_bind_root_dn" + // AuthenticationLDAPSASLBindRootPWD defines the password of the user to login the LDAP server and perform search. + AuthenticationLDAPSASLBindRootPWD = "authentication_ldap_sasl_bind_root_pwd" + // AuthenticationLDAPSASLInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSASLInitPoolSize = "authentication_ldap_sasl_init_pool_size" + // AuthenticationLDAPSASLMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSASLMaxPoolSize = "authentication_ldap_sasl_max_pool_size" + // AuthenticationLDAPSimpleAuthMethodName defines the authentication method used by LDAP Simple authentication plugin + AuthenticationLDAPSimpleAuthMethodName = "authentication_ldap_simple_auth_method_name" + // AuthenticationLDAPSimpleCAPath defines the ca certificate to verify LDAP connection in LDAP Simple authentication plugin + AuthenticationLDAPSimpleCAPath = "authentication_ldap_simple_ca_path" + // AuthenticationLDAPSimpleTLS defines whether to use TLS connection in LDAP Simple authentication plugin + AuthenticationLDAPSimpleTLS = "authentication_ldap_simple_tls" + // AuthenticationLDAPSimpleServerHost defines the server host of LDAP server for LDAP Simple authentication plugin + AuthenticationLDAPSimpleServerHost = "authentication_ldap_simple_server_host" + // AuthenticationLDAPSimpleServerPort defines the port of LDAP server for LDAP Simple authentication plugin + AuthenticationLDAPSimpleServerPort = "authentication_ldap_simple_server_port" + // AuthenticationLDAPSimpleReferral defines whether to enable LDAP referral for LDAP Simple authentication plugin + AuthenticationLDAPSimpleReferral = "authentication_ldap_simple_referral" + // AuthenticationLDAPSimpleUserSearchAttr defines the attribute of username in LDAP server + AuthenticationLDAPSimpleUserSearchAttr = "authentication_ldap_simple_user_search_attr" + // AuthenticationLDAPSimpleBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. + AuthenticationLDAPSimpleBindBaseDN = "authentication_ldap_simple_bind_base_dn" + // AuthenticationLDAPSimpleBindRootDN defines the `dn` of the user to login the LDAP server and perform search. + AuthenticationLDAPSimpleBindRootDN = "authentication_ldap_simple_bind_root_dn" + // AuthenticationLDAPSimpleBindRootPWD defines the password of the user to login the LDAP server and perform search. + AuthenticationLDAPSimpleBindRootPWD = "authentication_ldap_simple_bind_root_pwd" + // AuthenticationLDAPSimpleInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSimpleInitPoolSize = "authentication_ldap_simple_init_pool_size" + // AuthenticationLDAPSimpleMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. + AuthenticationLDAPSimpleMaxPoolSize = "authentication_ldap_simple_max_pool_size" + // TiDBRuntimeFilterTypeName the value of is string, a runtime filter type list split by ",", such as: "IN,MIN_MAX" + TiDBRuntimeFilterTypeName = "tidb_runtime_filter_type" + // TiDBRuntimeFilterModeName the mode of runtime filter, such as "OFF", "LOCAL" + TiDBRuntimeFilterModeName = "tidb_runtime_filter_mode" + // TiDBSkipMissingPartitionStats controls how to handle missing partition stats when merging partition stats to global stats. + // When set to true, skip missing partition stats and continue to merge other partition stats to global stats. + // When set to false, give up merging partition stats to global stats. + TiDBSkipMissingPartitionStats = "tidb_skip_missing_partition_stats" + // TiDBSessionAlias indicates the alias of a session which is used for tracing. + TiDBSessionAlias = "tidb_session_alias" + // TiDBServiceScope indicates the role for tidb for distributed task framework. + TiDBServiceScope = "tidb_service_scope" + // TiDBSchemaVersionCacheLimit defines the capacity size of domain infoSchema cache. + TiDBSchemaVersionCacheLimit = "tidb_schema_version_cache_limit" + // TiDBEnableTiFlashPipelineMode means if we should use pipeline model to execute query or not in tiflash. + // It's deprecated and setting it will not have any effect. + TiDBEnableTiFlashPipelineMode = "tidb_enable_tiflash_pipeline_model" + // TiDBIdleTransactionTimeout indicates the maximum time duration a transaction could be idle, unit is second. + // Any idle transaction will be killed after being idle for `tidb_idle_transaction_timeout` seconds. + // This is similar to https://docs.percona.com/percona-server/5.7/management/innodb_kill_idle_trx.html and https://mariadb.com/kb/en/transaction-timeouts/ + TiDBIdleTransactionTimeout = "tidb_idle_transaction_timeout" + // TiDBLowResolutionTSOUpdateInterval defines how often to refresh low resolution timestamps. + TiDBLowResolutionTSOUpdateInterval = "tidb_low_resolution_tso_update_interval" + // TiDBDMLType indicates the execution type of DML in TiDB. + // The value can be STANDARD, BULK. + // Currently, the BULK mode only affects auto-committed DML. + TiDBDMLType = "tidb_dml_type" + // TiFlashHashAggPreAggMode indicates the policy of 1st hashagg. + TiFlashHashAggPreAggMode = "tiflash_hashagg_preaggregation_mode" + // TiDBEnableLazyCursorFetch defines whether to enable the lazy cursor fetch. If it's `OFF`, all results of + // of a cursor will be stored in the tidb node in `EXECUTE` command. + TiDBEnableLazyCursorFetch = "tidb_enable_lazy_cursor_fetch" + // TiDBTSOClientRPCMode controls how the TSO client performs the TSO RPC requests. It internally controls the + // concurrency of the RPC. This variable provides an approach to tune the latency of getting timestamps from PD. + TiDBTSOClientRPCMode = "tidb_tso_client_rpc_mode" + // TiDBCircuitBreakerPDMetadataErrorRateThresholdPct variable is used to set percent of errors to trip the circuit breaker for get region calls to PD + // https://github.com/tikv/rfcs/blob/master/text/0115-circuit-breaker.md + TiDBCircuitBreakerPDMetadataErrorRateThresholdPct = "tidb_cb_pd_metadata_error_rate_threshold_pct" +) + +// TiDB intentional limits, can be raised in the future. +const ( + // MaxConfigurableConcurrency is the maximum number of "threads" (goroutines) that can be specified + // for any type of configuration item that has concurrent workers. + MaxConfigurableConcurrency = 256 + + // MaxShardRowIDBits is the maximum number of bits that can be used for row-id sharding. + MaxShardRowIDBits = 15 + + // MaxPreSplitRegions is the maximum number of regions that can be pre-split. + MaxPreSplitRegions = 15 +) + +// Default TiDB system variable values. +const ( + DefHostname = "localhost" + DefIndexLookupConcurrency = ConcurrencyUnset + DefIndexLookupJoinConcurrency = ConcurrencyUnset + DefIndexSerialScanConcurrency = 1 + DefIndexJoinBatchSize = 25000 + DefIndexLookupSize = 20000 + DefDistSQLScanConcurrency = 15 + DefAnalyzeDistSQLScanConcurrency = 4 + DefBuildStatsConcurrency = 2 + DefBuildSamplingStatsConcurrency = 2 + DefAutoAnalyzeRatio = 0.5 + DefAutoAnalyzeStartTime = "00:00 +0000" + DefAutoAnalyzeEndTime = "23:59 +0000" + DefAutoIncrementIncrement = 1 + DefAutoIncrementOffset = 1 + DefChecksumTableConcurrency = 4 + DefSkipUTF8Check = false + DefSkipASCIICheck = false + DefOptAggPushDown = false + DefOptDeriveTopN = false + DefOptCartesianBCJ = 1 + DefOptMPPOuterJoinFixedBuildSide = false + DefOptWriteRowID = false + DefOptEnableCorrelationAdjustment = true + DefOptLimitPushDownThreshold = 100 + DefOptCorrelationThreshold = 0.9 + DefOptCorrelationExpFactor = 1 + DefOptCPUFactor = 3.0 + DefOptCopCPUFactor = 3.0 + DefOptTiFlashConcurrencyFactor = 24.0 + DefOptNetworkFactor = 1.0 + DefOptScanFactor = 1.5 + DefOptDescScanFactor = 3.0 + DefOptSeekFactor = 20.0 + DefOptMemoryFactor = 0.001 + DefOptDiskFactor = 1.5 + DefOptConcurrencyFactor = 3.0 + DefOptForceInlineCTE = false + DefOptInSubqToJoinAndAgg = true + DefOptPreferRangeScan = true + DefBatchInsert = false + DefBatchDelete = false + DefBatchCommit = false + DefCurretTS = 0 + DefInitChunkSize = 32 + DefMinPagingSize = int(paging.MinPagingSize) + DefMaxPagingSize = int(paging.MaxPagingSize) + DefMaxChunkSize = 1024 + DefDMLBatchSize = 0 + DefMaxPreparedStmtCount = -1 + DefWaitTimeout = 28800 + DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. + DefTiDBMemQuotaBindingCache = 64 << 20 // 64MB. + DefTiDBGeneralLog = false + DefTiDBPProfSQLCPU = 0 + DefTiDBRetryLimit = 10 + DefTiDBDisableTxnAutoRetry = true + DefTiDBConstraintCheckInPlace = false + DefTiDBHashJoinConcurrency = ConcurrencyUnset + DefTiDBProjectionConcurrency = ConcurrencyUnset + DefBroadcastJoinThresholdSize = 100 * 1024 * 1024 + DefBroadcastJoinThresholdCount = 10 * 1024 + DefPreferBCJByExchangeDataSize = false + DefTiDBOptimizerSelectivityLevel = 0 + DefTiDBOptimizerEnableNewOFGB = false + DefTiDBEnableOuterJoinReorder = true + DefTiDBEnableNAAJ = true + DefTiDBAllowBatchCop = 1 + DefShardRowIDBits = 0 + DefPreSplitRegions = 0 + DefBlockEncryptionMode = "aes-128-ecb" + DefTiDBAllowMPPExecution = true + DefTiDBAllowTiFlashCop = false + DefTiDBHashExchangeWithNewCollation = true + DefTiDBEnforceMPPExecution = false + DefTiFlashMaxThreads = -1 + DefTiFlashMaxBytesBeforeExternalJoin = -1 + DefTiFlashMaxBytesBeforeExternalGroupBy = -1 + DefTiFlashMaxBytesBeforeExternalSort = -1 + DefTiFlashMemQuotaQueryPerNode = 0 + DefTiFlashQuerySpillRatio = 0.7 + DefTiDBEnableTiFlashPipelineMode = true + DefTiDBMPPStoreFailTTL = "60s" + DefTiDBTxnMode = PessimisticTxnMode + DefTiDBRowFormatV1 = 1 + DefTiDBRowFormatV2 = 2 + DefTiDBDDLReorgWorkerCount = 4 + DefTiDBDDLReorgBatchSize = 256 + DefTiDBDDLFlashbackConcurrency = 64 + DefTiDBDDLErrorCountLimit = 512 + DefTiDBDDLReorgMaxWriteSpeed = 0 + DefTiDBMaxDeltaSchemaCount = 1024 + DefTiDBPlacementMode = PlacementModeStrict + DefTiDBEnableAutoIncrementInGenerated = false + DefTiDBHashAggPartialConcurrency = ConcurrencyUnset + DefTiDBHashAggFinalConcurrency = ConcurrencyUnset + DefTiDBWindowConcurrency = ConcurrencyUnset + DefTiDBMergeJoinConcurrency = 1 // disable optimization by default + DefTiDBStreamAggConcurrency = 1 + DefTiDBForcePriority = mysql.NoPriority + DefEnableWindowFunction = true + DefEnablePipelinedWindowFunction = true + DefEnableStrictDoubleTypeCheck = true + DefEnableVectorizedExpression = true + DefTiDBOptJoinReorderThreshold = 0 + DefTiDBDDLSlowOprThreshold = 300 + DefTiDBUseFastAnalyze = false + DefTiDBSkipIsolationLevelCheck = false + DefTiDBExpensiveQueryTimeThreshold = 60 // 60s + DefTiDBExpensiveTxnTimeThreshold = 60 * 10 // 10 minutes + DefTiDBScatterRegion = ScatterOff + DefTiDBWaitSplitRegionFinish = true + DefWaitSplitRegionTimeout = 300 // 300s + DefTiDBEnableNoopFuncs = Off + DefTiDBEnableNoopVariables = true + DefTiDBAllowRemoveAutoInc = false + DefTiDBUsePlanBaselines = true + DefTiDBEvolvePlanBaselines = false + DefTiDBEvolvePlanTaskMaxTime = 600 // 600s + DefTiDBEvolvePlanTaskStartTime = "00:00 +0000" + DefTiDBEvolvePlanTaskEndTime = "23:59 +0000" + DefInnodbLockWaitTimeout = 50 // 50s + DefTiDBStoreLimit = 0 + DefTiDBMetricSchemaStep = 60 // 60s + DefTiDBMetricSchemaRangeDuration = 60 // 60s + DefTiDBFoundInPlanCache = false + DefTiDBFoundInBinding = false + DefTiDBEnableCollectExecutionInfo = true + DefTiDBAllowAutoRandExplicitInsert = false + DefTiDBEnableClusteredIndex = ClusteredIndexDefModeOn + DefTiDBRedactLog = Off + DefTiDBRestrictedReadOnly = false + DefTiDBSuperReadOnly = false + DefTiDBShardAllocateStep = math.MaxInt64 + DefTiDBEnableTelemetry = false + DefTiDBEnableParallelApply = false + DefTiDBPartitionPruneMode = "dynamic" + DefTiDBEnableRateLimitAction = false + DefTiDBEnableAsyncCommit = false + DefTiDBEnable1PC = false + DefTiDBGuaranteeLinearizability = true + DefTiDBAnalyzeVersion = 2 + // Deprecated: This variable is deprecated, please do not use this variable. + DefTiDBAutoAnalyzePartitionBatchSize = mysql.PartitionCountLimit + DefTiDBEnableIndexMergeJoin = false + DefTiDBTrackAggregateMemoryUsage = true + DefCTEMaxRecursionDepth = 1000 + DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. + DefTiDBEnableLocalTxn = false + DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms + DefTiDBEnableTSOFollowerProxy = false + DefPDEnableFollowerHandleRegion = true + DefTiDBEnableOrderedResultMode = false + DefTiDBEnablePseudoForOutdatedStats = false + DefTiDBRegardNULLAsPoint = true + DefEnablePlacementCheck = true + DefTimestamp = "0" + DefTimestampFloat = 0.0 + DefTiDBEnableStmtSummary = true + DefTiDBStmtSummaryInternalQuery = false + DefTiDBStmtSummaryRefreshInterval = 1800 + DefTiDBStmtSummaryHistorySize = 24 + DefTiDBStmtSummaryMaxStmtCount = 3000 + DefTiDBStmtSummaryMaxSQLLength = 4096 + DefTiDBCapturePlanBaseline = Off + DefTiDBIgnoreInlistPlanDigest = false + DefTiDBEnableIndexMerge = true + DefEnableLegacyInstanceScope = true + DefTiDBTableCacheLease = 3 // 3s + DefTiDBPersistAnalyzeOptions = true + DefTiDBStatsLoadSyncWait = 100 + DefTiDBStatsLoadPseudoTimeout = true + DefSysdateIsNow = false + DefTiDBEnableParallelHashaggSpill = true + DefTiDBEnableMutationChecker = false + DefTiDBTxnAssertionLevel = AssertionOffStr + DefTiDBIgnorePreparedCacheCloseStmt = false + DefTiDBBatchPendingTiFlashCount = 4000 + DefRCReadCheckTS = false + DefTiDBRemoveOrderbyInSubquery = true + DefTiDBSkewDistinctAgg = false + DefTiDB3StageDistinctAgg = true + DefTiDB3StageMultiDistinctAgg = false + DefTiDBOptExplainEvaledSubquery = false + DefTiDBReadStaleness = 0 + DefTiDBGCMaxWaitTime = 24 * 60 * 60 + DefMaxAllowedPacket uint64 = 67108864 + DefTiDBEnableBatchDML = false + DefTiDBMemQuotaQuery = memory.DefMemQuotaQuery // 1GB + DefTiDBStatsCacheMemQuota = 0 + MaxTiDBStatsCacheMemQuota = 1024 * 1024 * 1024 * 1024 // 1TB + DefTiDBQueryLogMaxLen = 4096 + DefRequireSecureTransport = false + DefTiDBCommitterConcurrency = 128 + DefTiDBBatchDMLIgnoreError = false + DefTiDBMemQuotaAnalyze = -1 + DefTiDBEnableAutoAnalyze = true + DefTiDBEnableAutoAnalyzePriorityQueue = true + DefTiDBAnalyzeColumnOptions = "PREDICATE" + DefTiDBMemOOMAction = "CANCEL" + DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 + DefTiDBAutoAnalyzeConcurrency = 1 + DefTiDBEnablePrepPlanCache = true + DefTiDBPrepPlanCacheSize = 100 + DefTiDBSessionPlanCacheSize = 100 + DefTiDBEnablePrepPlanCacheMemoryMonitor = true + DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 + DefTiDBEnableWorkloadBasedLearning = false + DefTiDBWorkloadBasedLearningInterval = 24 * time.Hour + DefTiDBEnableDistTask = true + DefTiDBEnableFastCreateTable = true + DefTiDBSimplifiedMetrics = false + DefTiDBEnablePaging = true + DefTiFlashFineGrainedShuffleStreamCount = 0 + DefStreamCountWhenMaxThreadsNotSet = 8 + DefTiFlashFineGrainedShuffleBatchSize = 8192 + DefAdaptiveClosestReadThreshold = 4096 + DefTiDBEnableAnalyzeSnapshot = false + DefTiDBGenerateBinaryPlan = true + DefEnableTiDBGCAwareMemoryTrack = false + DefTiDBDefaultStrMatchSelectivity = 0.8 + DefTiDBEnableTmpStorageOnOOM = true + DefTiDBEnableMDL = true + DefTiFlashFastScan = false + DefMemoryUsageAlarmRatio = 0.7 + DefMemoryUsageAlarmKeepRecordNum = 5 + DefTiDBEnableFastReorg = true + DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB + DefExecutorConcurrency = 5 + DefTiDBEnableNonPreparedPlanCache = false + DefTiDBEnableNonPreparedPlanCacheForDML = false + DefTiDBNonPreparedPlanCacheSize = 100 + DefTiDBPlanCacheMaxPlanSize = 2 * size.MB + DefTiDBInstancePlanCacheMaxMemSize = 100 * size.MB + MinTiDBInstancePlanCacheMemSize = 100 * size.MB + DefTiDBInstancePlanCacheReservedPercentage = 0.1 + // MaxDDLReorgBatchSize is exported for testing. + MaxDDLReorgBatchSize int32 = 10240 + MinDDLReorgBatchSize int32 = 32 + MinExpensiveQueryTimeThreshold uint64 = 10 // 10s + MinExpensiveTxnTimeThreshold uint64 = 60 // 60s + DefTiDBAutoBuildStatsConcurrency = 1 + DefTiDBSysProcScanConcurrency = 1 + DefTiDBRcWriteCheckTs = false + DefTiDBForeignKeyChecks = true + DefTiDBOptAdvancedJoinHint = true + DefTiDBAnalyzePartitionConcurrency = 2 + DefTiDBOptRangeMaxSize = 64 * int64(size.MB) // 64 MB + DefTiDBCostModelVer = 2 + DefTiDBServerMemoryLimitSessMinSize = 128 << 20 + DefTiDBMergePartitionStatsConcurrency = 1 + DefTiDBServerMemoryLimitGCTrigger = 0.7 + DefTiDBEnableGOGCTuner = true + // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. + DefTiDBGOGCTunerThreshold float64 = 0.6 + DefTiDBGOGCMaxValue = 500 + DefTiDBGOGCMinValue = 100 + DefTiDBOptPrefixIndexSingleScan = true + DefTiDBEnableAsyncMergeGlobalStats = true + DefTiDBExternalTS = 0 + DefTiDBEnableExternalTSRead = false + DefTiDBEnableReusechunk = true + DefTiDBUseAlloc = false + DefTiDBEnablePlanReplayerCapture = true + DefTiDBIndexMergeIntersectionConcurrency = ConcurrencyUnset + DefTiDBTTLJobEnable = true + DefTiDBTTLScanBatchSize = 500 + DefTiDBTTLScanBatchMaxSize = 10240 + DefTiDBTTLScanBatchMinSize = 1 + DefTiDBTTLDeleteBatchSize = 100 + DefTiDBTTLDeleteBatchMaxSize = 10240 + DefTiDBTTLDeleteBatchMinSize = 1 + DefTiDBTTLDeleteRateLimit = 0 + DefTiDBTTLRunningTasks = -1 + DefPasswordReuseHistory = 0 + DefPasswordReuseTime = 0 + DefTiDBStoreBatchSize = 4 + DefTiDBHistoricalStatsDuration = 7 * 24 * time.Hour + DefTiDBEnableHistoricalStatsForCapture = false + DefTiDBTTLJobScheduleWindowStartTime = "00:00 +0000" + DefTiDBTTLJobScheduleWindowEndTime = "23:59 +0000" + DefTiDBTTLScanWorkerCount = 4 + DefTiDBTTLDeleteWorkerCount = 4 + DefaultExchangeCompressionMode = ExchangeCompressionModeUnspecified + DefTiDBEnableResourceControl = true + DefTiDBResourceControlStrictMode = true + DefTiDBPessimisticTransactionFairLocking = false + DefTiDBEnablePlanCacheForParamLimit = true + DefTiDBEnableINLJoinMultiPattern = true + DefTiFlashComputeDispatchPolicy = DispatchPolicyConsistentHashStr + DefTiDBEnablePlanCacheForSubquery = true + DefTiDBLoadBasedReplicaReadThreshold = time.Second + DefTiDBOptEnableLateMaterialization = true + DefTiDBOptOrderingIdxSelThresh = 0.0 + DefTiDBOptOrderingIdxSelRatio = -1 + DefTiDBOptEnableMPPSharedCTEExecution = false + DefTiDBPlanCacheInvalidationOnFreshStats = true + DefTiDBEnableRowLevelChecksum = false + DefAuthenticationLDAPSASLAuthMethodName = "SCRAM-SHA-1" + DefAuthenticationLDAPSASLServerPort = 389 + DefAuthenticationLDAPSASLTLS = false + DefAuthenticationLDAPSASLUserSearchAttr = "uid" + DefAuthenticationLDAPSASLInitPoolSize = 10 + DefAuthenticationLDAPSASLMaxPoolSize = 1000 + DefAuthenticationLDAPSimpleAuthMethodName = "SIMPLE" + DefAuthenticationLDAPSimpleServerPort = 389 + DefAuthenticationLDAPSimpleTLS = false + DefAuthenticationLDAPSimpleUserSearchAttr = "uid" + DefAuthenticationLDAPSimpleInitPoolSize = 10 + DefAuthenticationLDAPSimpleMaxPoolSize = 1000 + DefTiFlashReplicaRead = AllReplicaStr + DefTiDBEnableFastCheckTable = true + DefRuntimeFilterType = "IN" + DefRuntimeFilterMode = "OFF" + DefTiDBLockUnchangedKeys = true + DefTiDBEnableCheckConstraint = false + DefTiDBSkipMissingPartitionStats = true + DefTiDBOptEnableHashJoin = true + DefTiDBHashJoinVersion = joinversion.HashJoinVersionOptimized + DefTiDBOptObjective = OptObjectiveModerate + DefTiDBSchemaVersionCacheLimit = 16 + DefTiDBIdleTransactionTimeout = 0 + DefTiDBTxnEntrySizeLimit = 0 + DefTiDBSchemaCacheSize = 512 * 1024 * 1024 + DefTiDBLowResolutionTSOUpdateInterval = 2000 + DefDivPrecisionIncrement = 4 + DefTiDBDMLType = "STANDARD" + DefGroupConcatMaxLen = uint64(1024) + DefDefaultWeekFormat = "0" + DefTiFlashPreAggMode = ForcePreAggStr + DefTiDBEnableLazyCursorFetch = false + DefOptEnableProjectionPushDown = true + DefTiDBEnableSharedLockPromotion = false + DefTiDBTSOClientRPCMode = TSOClientRPCModeDefault + DefTiDBCircuitBreakerPDMetaErrorRatePct = 0 + DefTiDBAccelerateUserCreationUpdate = false +) + +// Process global variables. +var ( + ProcessGeneralLog = atomic.NewBool(false) + RunAutoAnalyze = atomic.NewBool(DefTiDBEnableAutoAnalyze) + EnableAutoAnalyzePriorityQueue = atomic.NewBool(DefTiDBEnableAutoAnalyzePriorityQueue) + // AnalyzeColumnOptions is a global variable that indicates the default column choice for ANALYZE. + // The value of this variable is a string that can be one of the following values: + // "PREDICATE", "ALL". + // The behavior of the analyze operation depends on the value of `tidb_persist_analyze_options`: + // 1. If `tidb_persist_analyze_options` is enabled and the column choice from the analyze options record is set to `default`, + // the value of `tidb_analyze_column_options` determines the behavior of the analyze operation. + // 2. If `tidb_persist_analyze_options` is disabled, `tidb_analyze_column_options` is used directly to decide + // whether to analyze all columns or just the predicate columns. + AnalyzeColumnOptions = atomic.NewString(DefTiDBAnalyzeColumnOptions) + GlobalLogMaxDays = atomic.NewInt32(int32(config.GetGlobalConfig().Log.File.MaxDays)) + QueryLogMaxLen = atomic.NewInt32(DefTiDBQueryLogMaxLen) + EnablePProfSQLCPU = atomic.NewBool(false) + EnableBatchDML = atomic.NewBool(false) + EnableTmpStorageOnOOM = atomic.NewBool(DefTiDBEnableTmpStorageOnOOM) + DDLReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount + DDLReorgBatchSize int32 = DefTiDBDDLReorgBatchSize + DDLFlashbackConcurrency int32 = DefTiDBDDLFlashbackConcurrency + DDLErrorCountLimit int64 = DefTiDBDDLErrorCountLimit + DDLReorgRowFormat int64 = DefTiDBRowFormatV2 + DDLReorgMaxWriteSpeed = atomic.NewInt64(DefTiDBDDLReorgMaxWriteSpeed) + MaxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount + // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. + DDLSlowOprThreshold = config.GetGlobalConfig().Instance.DDLSlowOprThreshold + ForcePriority = int32(DefTiDBForcePriority) + MaxOfMaxAllowedPacket uint64 = 1073741824 + ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold + ExpensiveTxnTimeThreshold uint64 = DefTiDBExpensiveTxnTimeThreshold + MemoryUsageAlarmRatio = atomic.NewFloat64(DefMemoryUsageAlarmRatio) + MemoryUsageAlarmKeepRecordNum = atomic.NewInt64(DefMemoryUsageAlarmKeepRecordNum) + EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) + MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) + EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) + EnablePDFollowerHandleRegion = atomic.NewBool(DefPDEnableFollowerHandleRegion) + RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) + VarTiDBSuperReadOnly = atomic.NewBool(DefTiDBSuperReadOnly) + PersistAnalyzeOptions = atomic.NewBool(DefTiDBPersistAnalyzeOptions) + TableCacheLease = atomic.NewInt64(DefTiDBTableCacheLease) + StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) + StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) + MemQuotaBindingCache = atomic.NewInt64(DefTiDBMemQuotaBindingCache) + GCMaxWaitTime = atomic.NewInt64(DefTiDBGCMaxWaitTime) + StatsCacheMemQuota = atomic.NewInt64(DefTiDBStatsCacheMemQuota) + OOMAction = atomic.NewString(DefTiDBMemOOMAction) + MaxAutoAnalyzeTime = atomic.NewInt64(DefTiDBMaxAutoAnalyzeTime) + // variables for plan cache + PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio) + EnableInstancePlanCache = atomic.NewBool(false) + InstancePlanCacheReservedPercentage = atomic.NewFloat64(0.1) + InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) + EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) + EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) + EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) + EnableMDL = atomic.NewBool(false) + AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize) + AutoAnalyzeConcurrency = atomic.NewInt32(DefTiDBAutoAnalyzeConcurrency) + // TODO: set value by session variable + EnableWorkloadBasedLearning = atomic.NewBool(DefTiDBEnableWorkloadBasedLearning) + WorkloadBasedLearningInterval = atomic.NewDuration(DefTiDBWorkloadBasedLearningInterval) + // EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance. + EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg) + // DDLDiskQuota is the temporary variable for set disk quota for lightning + DDLDiskQuota = atomic.NewUint64(DefTiDBDDLDiskQuota) + // EnableForeignKey indicates whether to enable foreign key feature. + EnableForeignKey = atomic.NewBool(true) + EnableRCReadCheckTS = atomic.NewBool(false) + // EnableRowLevelChecksum indicates whether to append checksum to row values. + EnableRowLevelChecksum = atomic.NewBool(DefTiDBEnableRowLevelChecksum) + LowResolutionTSOUpdateInterval = atomic.NewUint32(DefTiDBLowResolutionTSOUpdateInterval) + + // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). + // It should be a const and shouldn't be modified after tidb is started. + DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() + GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) + PasswordValidationLength = atomic.NewInt32(8) + PasswordValidationMixedCaseCount = atomic.NewInt32(1) + PasswordValidtaionNumberCount = atomic.NewInt32(1) + PasswordValidationSpecialCharCount = atomic.NewInt32(1) + EnableTTLJob = atomic.NewBool(DefTiDBTTLJobEnable) + TTLScanBatchSize = atomic.NewInt64(DefTiDBTTLScanBatchSize) + TTLDeleteBatchSize = atomic.NewInt64(DefTiDBTTLDeleteBatchSize) + TTLDeleteRateLimit = atomic.NewInt64(DefTiDBTTLDeleteRateLimit) + TTLJobScheduleWindowStartTime = atomic.NewTime( + mustParseTime( + FullDayTimeFormat, + DefTiDBTTLJobScheduleWindowStartTime, + ), + ) + TTLJobScheduleWindowEndTime = atomic.NewTime( + mustParseTime( + FullDayTimeFormat, + DefTiDBTTLJobScheduleWindowEndTime, + ), + ) + TTLScanWorkerCount = atomic.NewInt32(DefTiDBTTLScanWorkerCount) + TTLDeleteWorkerCount = atomic.NewInt32(DefTiDBTTLDeleteWorkerCount) + PasswordHistory = atomic.NewInt64(DefPasswordReuseHistory) + PasswordReuseInterval = atomic.NewInt64(DefPasswordReuseTime) + IsSandBoxModeEnabled = atomic.NewBool(false) + MaxPreparedStmtCountValue = atomic.NewInt64(DefMaxPreparedStmtCount) + HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration) + EnableHistoricalStatsForCapture = atomic.NewBool(DefTiDBEnableHistoricalStatsForCapture) + TTLRunningTasks = atomic.NewInt32(DefTiDBTTLRunningTasks) + // always set the default value to false because the resource control in kv-client is not inited + // It will be initialized to the right value after the first call of `rebuildSysVarCache` + EnableResourceControl = atomic.NewBool(false) + EnableResourceControlStrictMode = atomic.NewBool(true) + EnableCheckConstraint = atomic.NewBool(DefTiDBEnableCheckConstraint) + SkipMissingPartitionStats = atomic.NewBool(DefTiDBSkipMissingPartitionStats) + TiFlashEnablePipelineMode = atomic.NewBool(DefTiDBEnableTiFlashPipelineMode) + ServiceScope = atomic.NewString("") + SchemaVersionCacheLimit = atomic.NewInt64(DefTiDBSchemaVersionCacheLimit) + CloudStorageURI = atomic.NewString("") + IgnoreInlistPlanDigest = atomic.NewBool(DefTiDBIgnoreInlistPlanDigest) + TxnEntrySizeLimit = atomic.NewUint64(DefTiDBTxnEntrySizeLimit) + + SchemaCacheSize = atomic.NewUint64(DefTiDBSchemaCacheSize) + SchemaCacheSizeOriginText = atomic.NewString(strconv.Itoa(DefTiDBSchemaCacheSize)) + AccelerateUserCreationUpdate = atomic.NewBool(DefTiDBAccelerateUserCreationUpdate) +) + +func serverMemoryLimitDefaultValue() string { + total, err := memory.MemTotal() + if err == nil && total != 0 { + return "80%" + } + return "0" +} + +func mustParseDuration(str string) time.Duration { + duration, err := time.ParseDuration(str) + if err != nil { + panic(fmt.Sprintf("%s is not a duration", str)) + } + + return duration +} + +func mustParseTime(layout string, str string) time.Time { + time, err := time.ParseInLocation(layout, str, time.UTC) + if err != nil { + panic(fmt.Sprintf("%s is not in %s duration format", str, layout)) + } + + return time +} + +const ( + // OptObjectiveModerate is a possible value and the default value for TiDBOptObjective. + // Please see comments of SessionVars.OptObjective for details. + OptObjectiveModerate string = "moderate" + // OptObjectiveDeterminate is a possible value for TiDBOptObjective. + OptObjectiveDeterminate = "determinate" +) + +// ForcePreAggStr means 1st hashagg will be pre aggregated. +// AutoStr means TiFlash will decide which policy for 1st hashagg. +// ForceStreamingStr means 1st hashagg will for pass through all blocks. +const ( + ForcePreAggStr = "force_preagg" + AutoStr = "auto" + ForceStreamingStr = "force_streaming" +) + +const ( + // AllReplicaStr is the string value of AllReplicas. + AllReplicaStr = "all_replicas" + // ClosestAdaptiveStr is the string value of ClosestAdaptive. + ClosestAdaptiveStr = "closest_adaptive" + // ClosestReplicasStr is the string value of ClosestReplicas. + ClosestReplicasStr = "closest_replicas" +) + +const ( + // DispatchPolicyRRStr is string value for DispatchPolicyRR. + DispatchPolicyRRStr = "round_robin" + // DispatchPolicyConsistentHashStr is string value for DispatchPolicyConsistentHash. + DispatchPolicyConsistentHashStr = "consistent_hash" + // DispatchPolicyInvalidStr is string value for DispatchPolicyInvalid. + DispatchPolicyInvalidStr = "invalid" +) + +// ConcurrencyUnset means the value the of the concurrency related variable is unset. +const ConcurrencyUnset = -1 + +// ExchangeCompressionMode means the compress method used in exchange operator +type ExchangeCompressionMode int + +const ( + // ExchangeCompressionModeNONE indicates no compression + ExchangeCompressionModeNONE ExchangeCompressionMode = iota + // ExchangeCompressionModeFast indicates fast compression/decompression speed, compression ratio is lower than HC mode + ExchangeCompressionModeFast + // ExchangeCompressionModeHC indicates high compression (HC) ratio mode + ExchangeCompressionModeHC + // ExchangeCompressionModeUnspecified indicates unspecified compress method, let TiDB choose one + ExchangeCompressionModeUnspecified + + // RecommendedExchangeCompressionMode indicates recommended compression mode + RecommendedExchangeCompressionMode ExchangeCompressionMode = ExchangeCompressionModeFast + + exchangeCompressionModeUnspecifiedName string = "UNSPECIFIED" +) + +// Name returns the name of ExchangeCompressionMode +func (t ExchangeCompressionMode) Name() string { + if t == ExchangeCompressionModeUnspecified { + return exchangeCompressionModeUnspecifiedName + } + return t.ToTipbCompressionMode().String() +} + +// ToExchangeCompressionMode returns the ExchangeCompressionMode from name +func ToExchangeCompressionMode(name string) (ExchangeCompressionMode, bool) { + name = strings.ToUpper(name) + if name == exchangeCompressionModeUnspecifiedName { + return ExchangeCompressionModeUnspecified, true + } + value, ok := tipb.CompressionMode_value[name] + if ok { + return ExchangeCompressionMode(value), true + } + return ExchangeCompressionModeNONE, false +} + +// ToTipbCompressionMode returns tipb.CompressionMode from kv.ExchangeCompressionMode +func (t ExchangeCompressionMode) ToTipbCompressionMode() tipb.CompressionMode { + switch t { + case ExchangeCompressionModeNONE: + return tipb.CompressionMode_NONE + case ExchangeCompressionModeFast: + return tipb.CompressionMode_FAST + case ExchangeCompressionModeHC: + return tipb.CompressionMode_HIGH_COMPRESSION + } + return tipb.CompressionMode_NONE +} + +// ScopeFlag is for system variable whether can be changed in global/session dynamically or not. +type ScopeFlag uint8 + +// TypeFlag is the SysVar type, which doesn't exactly match MySQL types. +type TypeFlag byte + +const ( + // ScopeNone means the system variable can not be changed dynamically. + ScopeNone ScopeFlag = 0 + // ScopeGlobal means the system variable can be changed globally. + ScopeGlobal ScopeFlag = 1 << 0 + // ScopeSession means the system variable can only be changed in current session. + ScopeSession ScopeFlag = 1 << 1 + // ScopeInstance means it is similar to global but doesn't propagate to other TiDB servers. + ScopeInstance ScopeFlag = 1 << 2 + + // TypeStr is the default + TypeStr TypeFlag = iota + // TypeBool for boolean + TypeBool + // TypeInt for integer + TypeInt + // TypeEnum for Enum + TypeEnum + // TypeFloat for Double + TypeFloat + // TypeUnsigned for Unsigned integer + TypeUnsigned + // TypeTime for time of day (a TiDB extension) + TypeTime + // TypeDuration for a golang duration (a TiDB extension) + TypeDuration + + // On is the canonical string for ON + On = "ON" + // Off is the canonical string for OFF + Off = "OFF" + // Warn means return warnings + Warn = "WARN" + // IntOnly means enable for int type + IntOnly = "INT_ONLY" + // Marker is a special log redact behavior + Marker = "MARKER" + + // AssertionStrictStr is a choice of variable TiDBTxnAssertionLevel that means full assertions should be performed, + // even if the performance might be slowed down. + AssertionStrictStr = "STRICT" + // AssertionFastStr is a choice of variable TiDBTxnAssertionLevel that means assertions that doesn't affect + // performance should be performed. + AssertionFastStr = "FAST" + // AssertionOffStr is a choice of variable TiDBTxnAssertionLevel that means no assertion should be performed. + AssertionOffStr = "OFF" + // OOMActionCancel constants represents the valid action configurations for OOMAction "CANCEL". + OOMActionCancel = "CANCEL" + // OOMActionLog constants represents the valid action configurations for OOMAction "LOG". + OOMActionLog = "LOG" + + // TSOClientRPCModeDefault is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client sends batched + // TSO requests serially. + TSOClientRPCModeDefault = "DEFAULT" + // TSOClientRPCModeParallel is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to + // keep approximately 2 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time + // by half, at the expense of about twice the amount of TSO RPC calls. + TSOClientRPCModeParallel = "PARALLEL" + // TSOClientRPCModeParallelFast is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to + // keep approximately 4 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time + // by 3/4, at the expense of about 4 times the amount of TSO RPC calls. + TSOClientRPCModeParallelFast = "PARALLEL-FAST" +) + +// Global config name list. +const ( + GlobalConfigEnableTopSQL = "enable_resource_metering" + GlobalConfigSourceID = "source_id" +) + +func (s ScopeFlag) String() string { + var scopes []string + if s == ScopeNone { + return "NONE" + } + if s&ScopeSession != 0 { + scopes = append(scopes, "SESSION") + } + if s&ScopeGlobal != 0 { + scopes = append(scopes, "GLOBAL") + } + if s&ScopeInstance != 0 { + scopes = append(scopes, "INSTANCE") + } + return strings.Join(scopes, ",") +} + +// ClusteredIndexDefMode controls the default clustered property for primary key. +type ClusteredIndexDefMode int + +const ( + // ClusteredIndexDefModeIntOnly indicates only single int primary key will default be clustered. + ClusteredIndexDefModeIntOnly ClusteredIndexDefMode = 0 + // ClusteredIndexDefModeOn indicates primary key will default be clustered. + ClusteredIndexDefModeOn ClusteredIndexDefMode = 1 + // ClusteredIndexDefModeOff indicates primary key will default be non-clustered. + ClusteredIndexDefModeOff ClusteredIndexDefMode = 2 +) + +// TiDBOptEnableClustered converts enable clustered options to ClusteredIndexDefMode. +func TiDBOptEnableClustered(opt string) ClusteredIndexDefMode { + switch opt { + case On: + return ClusteredIndexDefModeOn + case Off: + return ClusteredIndexDefModeOff + default: + return ClusteredIndexDefModeIntOnly + } +} + +const ( + // ScatterOff means default, will not scatter region + ScatterOff string = "" + // ScatterTable means scatter region at table level + ScatterTable string = "table" + // ScatterGlobal means scatter region at global level + ScatterGlobal string = "global" +) + +const ( + // PlacementModeStrict indicates all placement operations should be checked strictly in ddl + PlacementModeStrict string = "STRICT" + // PlacementModeIgnore indicates ignore all placement operations in ddl + PlacementModeIgnore string = "IGNORE" +) + +const ( + // LocalDayTimeFormat is the local format of analyze start time and end time. + LocalDayTimeFormat = "15:04" + // FullDayTimeFormat is the full format of analyze start time and end time. + FullDayTimeFormat = "15:04 -0700" +) + +// SetDDLReorgWorkerCounter sets DDLReorgWorkerCounter count. +// Sysvar validation enforces the range to already be correct. +func SetDDLReorgWorkerCounter(cnt int32) { + goatomic.StoreInt32(&DDLReorgWorkerCounter, cnt) +} + +// GetDDLReorgWorkerCounter gets DDLReorgWorkerCounter. +func GetDDLReorgWorkerCounter() int32 { + return goatomic.LoadInt32(&DDLReorgWorkerCounter) +} + +// SetDDLFlashbackConcurrency sets DDLFlashbackConcurrency count. +// Sysvar validation enforces the range to already be correct. +func SetDDLFlashbackConcurrency(cnt int32) { + goatomic.StoreInt32(&DDLFlashbackConcurrency, cnt) +} + +// GetDDLFlashbackConcurrency gets DDLFlashbackConcurrency count. +func GetDDLFlashbackConcurrency() int32 { + return goatomic.LoadInt32(&DDLFlashbackConcurrency) +} + +// SetDDLReorgBatchSize sets DDLReorgBatchSize size. +// Sysvar validation enforces the range to already be correct. +func SetDDLReorgBatchSize(cnt int32) { + goatomic.StoreInt32(&DDLReorgBatchSize, cnt) +} + +// GetDDLReorgBatchSize gets DDLReorgBatchSize. +func GetDDLReorgBatchSize() int32 { + return goatomic.LoadInt32(&DDLReorgBatchSize) +} + +// SetDDLErrorCountLimit sets ddlErrorCountlimit size. +func SetDDLErrorCountLimit(cnt int64) { + goatomic.StoreInt64(&DDLErrorCountLimit, cnt) +} + +// GetDDLErrorCountLimit gets ddlErrorCountlimit size. +func GetDDLErrorCountLimit() int64 { + return goatomic.LoadInt64(&DDLErrorCountLimit) +} + +// SetDDLReorgRowFormat sets DDLReorgRowFormat version. +func SetDDLReorgRowFormat(format int64) { + goatomic.StoreInt64(&DDLReorgRowFormat, format) +} + +// GetDDLReorgRowFormat gets DDLReorgRowFormat version. +func GetDDLReorgRowFormat() int64 { + return goatomic.LoadInt64(&DDLReorgRowFormat) +} + +// SetMaxDeltaSchemaCount sets MaxDeltaSchemaCount size. +func SetMaxDeltaSchemaCount(cnt int64) { + goatomic.StoreInt64(&MaxDeltaSchemaCount, cnt) +} + +// GetMaxDeltaSchemaCount gets MaxDeltaSchemaCount size. +func GetMaxDeltaSchemaCount() int64 { + return goatomic.LoadInt64(&MaxDeltaSchemaCount) +} diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index 3d691f9fede86..238dde11f99d2 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -3584,6 +3584,12 @@ var defaultSysVars = []*SysVar{ return nil }, }, + {Scope: ScopeGlobal, Name: TiDBAccelerateUserCreationUpdate, Value: BoolToOnOff(DefTiDBAccelerateUserCreationUpdate), Type: TypeBool, + SetGlobal: func(_ context.Context, s *SessionVars, val string) error { + AccelerateUserCreationUpdate.Store(TiDBOptOn(val)) + return nil + }, + }, } // GlobalSystemVariableInitialValue gets the default value for a system variable including ones that are dynamically set (e.g. based on the store) From 0339f9160d70e2777fd3330294b56b837d38a515 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 27 Feb 2025 14:54:06 +0800 Subject: [PATCH 05/17] privilege, domain: reduce the memory jitter of privilege reload activity for 2M users (#59487) ref pingcap/tidb#55563, close pingcap/tidb#59403 --- pkg/privilege/privileges/cache.go | 857 ++++++++++++++----------- pkg/privilege/privileges/cache_test.go | 60 +- pkg/privilege/privileges/privileges.go | 2 +- 3 files changed, 498 insertions(+), 421 deletions(-) diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 48b521e7a1425..029ec12d23fb5 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -444,12 +444,15 @@ func findRole(h *Handle, user string, host string, role *auth.RoleIdentity) bool } // LoadAll loads the tables from database to memory. -func (p *MySQLPrivilege) LoadAll(ctx sqlexec.RestrictedSQLExecutor) error { +func (p *MySQLPrivilege) LoadAll(ctx sqlexec.SQLExecutor) error { err := p.LoadUserTable(ctx) if err != nil { logutil.BgLogger().Warn("load mysql.user fail", zap.Error(err)) return errLoadPrivilege.FastGen("mysql.user") } + if l := p.user.Len(); l > 1024 { + logutil.BgLogger().Warn("load all called and user list is long, suggest enabling @@global.tidb_accelerate_user_creation_update", zap.Int("len", l)) + } err = p.LoadGlobalPrivTable(ctx) if err != nil { @@ -508,171 +511,159 @@ func (p *MySQLPrivilege) LoadAll(ctx sqlexec.RestrictedSQLExecutor) error { return nil } -func findUserAndAllRoles(all map[string]struct{}, roleGraph map[string]roleGraphEdgesTable) { - for { - before := len(all) +func findUserAndAllRoles(userList []string, roleGraph map[auth.RoleIdentity]roleGraphEdgesTable) map[string]struct{} { + // Including the user list and also their roles + all := make(map[string]struct{}, len(userList)) + queue := make([]string, 0, len(userList)) + + // Initialize the queue with the initial user list + for _, user := range userList { + all[user] = struct{}{} + queue = append(queue, user) + } + // Process the queue using BFS + for len(queue) > 0 { + user := queue[0] + queue = queue[1:] for userHost, value := range roleGraph { - user, _, found := strings.Cut(userHost, "@") - if !found { - // this should never happen - continue - } - if _, ok := all[user]; ok { - // If a user is in map, all its role should also added + if userHost.Username == user { for _, role := range value.roleList { - all[role.Username] = struct{}{} + if _, ok := all[role.Username]; !ok { + all[role.Username] = struct{}{} + queue = append(queue, role.Username) + } } } } - - // loop until the map does not expand - after := len(all) - if before == after { - break - } } + return all } -func (p *MySQLPrivilege) loadSomeUsers(ctx sqlexec.RestrictedSQLExecutor, userList ...string) ([]string, error) { - if len(userList) > 512 { - logutil.BgLogger().Warn("loadSomeUsers called with a long user list", zap.Int("len", len(userList))) - } - // Load the full role edge table first. - p.roleGraph = make(map[string]roleGraphEdgesTable) - err := p.loadTable(ctx, sqlLoadRoleGraph, p.decodeRoleEdgesTable) +func (p *MySQLPrivilege) loadSomeUsers(ctx sqlexec.SQLExecutor, userList map[string]struct{}) error { + err := loadTable(ctx, addUserFilterCondition(sqlLoadUserTable, userList), p.decodeUserTableRow(userList)) if err != nil { - return nil, errors.Trace(err) - } - - // Including the user list and also their roles - extendedUserList := make(map[string]struct{}, len(userList)) - for _, user := range userList { - extendedUserList[user] = struct{}{} - } - findUserAndAllRoles(extendedUserList, p.roleGraph) - // Re-generate the user list. - userList = userList[:0] - for user := range extendedUserList { - userList = append(userList, user) - } - - err = p.loadTable(ctx, sqlLoadUserTable, p.decodeUserTableRow, userList...) - if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - err = p.loadTable(ctx, sqlLoadGlobalPrivTable, p.decodeGlobalPrivTableRow, userList...) + err = loadTable(ctx, addUserFilterCondition(sqlLoadGlobalPrivTable, userList), p.decodeGlobalPrivTableRow(userList)) if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - err = p.loadTable(ctx, sqlLoadGlobalGrantsTable, p.decodeGlobalGrantsTableRow, userList...) + err = loadTable(ctx, addUserFilterCondition(sqlLoadGlobalGrantsTable, userList), p.decodeGlobalGrantsTableRow(userList)) if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - err = p.loadTable(ctx, sqlLoadDBTable, p.decodeDBTableRow, userList...) + err = loadTable(ctx, addUserFilterCondition(sqlLoadDBTable, userList), p.decodeDBTableRow(userList)) if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - err = p.loadTable(ctx, sqlLoadTablePrivTable, p.decodeTablesPrivTableRow, userList...) + err = loadTable(ctx, addUserFilterCondition(sqlLoadTablePrivTable, userList), p.decodeTablesPrivTableRow(userList)) if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - err = p.loadTable(ctx, sqlLoadDefaultRoles, p.decodeDefaultRoleTableRow, userList...) + err = loadTable(ctx, addUserFilterCondition(sqlLoadDefaultRoles, userList), p.decodeDefaultRoleTableRow(userList)) if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - err = p.loadTable(ctx, sqlLoadColumnsPrivTable, p.decodeColumnsPrivTableRow, userList...) + err = loadTable(ctx, addUserFilterCondition(sqlLoadColumnsPrivTable, userList), p.decodeColumnsPrivTableRow(userList)) if err != nil { - return nil, errors.Trace(err) + return errors.Trace(err) } - return userList, nil + return nil } -// merge construct a new MySQLPrivilege by merging the data of the two objects;. -func (p *MySQLPrivilege) merge(diff *MySQLPrivilege, userList []string) *MySQLPrivilege { +// merge construct a new MySQLPrivilege by merging the data of the two objects. +func (p *MySQLPrivilege) merge(diff *MySQLPrivilege, userList map[string]struct{}) *MySQLPrivilege { ret := newMySQLPrivilege() user := p.user.Clone() - for _, u := range userList { - user.Delete(itemUser{username: u}) + for u := range userList { + itm, ok := diff.user.Get(itemUser{username: u}) + if !ok { + user.Delete(itemUser{username: u}) + } else { + slices.SortFunc(itm.data, compareUserRecord) + user.ReplaceOrInsert(itm) + } } - diff.user.Ascend(func(itm itemUser) bool { - slices.SortFunc(itm.data, compareUserRecord) - user.ReplaceOrInsert(itm) - return true - }) ret.user.BTreeG = user db := p.db.Clone() - for _, u := range userList { - db.Delete(itemDB{username: u}) + for u := range userList { + itm, ok := diff.db.Get(itemDB{username: u}) + if !ok { + db.Delete(itemDB{username: u}) + } else { + slices.SortFunc(itm.data, compareDBRecord) + db.ReplaceOrInsert(itm) + } } - diff.db.Ascend(func(itm itemDB) bool { - slices.SortFunc(itm.data, compareDBRecord) - db.ReplaceOrInsert(itm) - return true - }) ret.db.BTreeG = db tablesPriv := p.tablesPriv.Clone() - for _, u := range userList { - tablesPriv.Delete(itemTablesPriv{username: u}) + for u := range userList { + itm, ok := diff.tablesPriv.Get(itemTablesPriv{username: u}) + if !ok { + tablesPriv.Delete(itemTablesPriv{username: u}) + } else { + slices.SortFunc(itm.data, compareTablesPrivRecord) + tablesPriv.ReplaceOrInsert(itm) + } } - diff.tablesPriv.Ascend(func(itm itemTablesPriv) bool { - slices.SortFunc(itm.data, compareTablesPrivRecord) - tablesPriv.ReplaceOrInsert(itm) - return true - }) ret.tablesPriv.BTreeG = tablesPriv columnsPriv := p.columnsPriv.Clone() - for _, u := range userList { - columnsPriv.Delete(itemColumnsPriv{username: u}) + for u := range userList { + itm, ok := diff.columnsPriv.Get(itemColumnsPriv{username: u}) + if !ok { + columnsPriv.Delete(itemColumnsPriv{username: u}) + } else { + slices.SortFunc(itm.data, compareColumnsPrivRecord) + columnsPriv.ReplaceOrInsert(itm) + } } - diff.columnsPriv.Ascend(func(itm itemColumnsPriv) bool { - slices.SortFunc(itm.data, compareColumnsPrivRecord) - columnsPriv.ReplaceOrInsert(itm) - return true - }) ret.columnsPriv.BTreeG = columnsPriv defaultRoles := p.defaultRoles.Clone() - for _, u := range userList { - defaultRoles.Delete(itemDefaultRole{username: u}) + for u := range userList { + itm, ok := diff.defaultRoles.Get(itemDefaultRole{username: u}) + if !ok { + defaultRoles.Delete(itemDefaultRole{username: u}) + } else { + slices.SortFunc(itm.data, compareDefaultRoleRecord) + defaultRoles.ReplaceOrInsert(itm) + } } - diff.defaultRoles.Ascend(func(itm itemDefaultRole) bool { - slices.SortFunc(itm.data, compareDefaultRoleRecord) - defaultRoles.ReplaceOrInsert(itm) - return true - }) ret.defaultRoles.BTreeG = defaultRoles dynamicPriv := p.dynamicPriv.Clone() - for _, u := range userList { - dynamicPriv.Delete(itemDynamicPriv{username: u}) + for u := range userList { + itm, ok := diff.dynamicPriv.Get(itemDynamicPriv{username: u}) + if !ok { + dynamicPriv.Delete(itemDynamicPriv{username: u}) + } else { + slices.SortFunc(itm.data, compareDynamicPrivRecord) + dynamicPriv.ReplaceOrInsert(itm) + } } - diff.dynamicPriv.Ascend(func(itm itemDynamicPriv) bool { - slices.SortFunc(itm.data, compareDynamicPrivRecord) - dynamicPriv.ReplaceOrInsert(itm) - return true - }) ret.dynamicPriv.BTreeG = dynamicPriv globalPriv := p.globalPriv.Clone() - for _, u := range userList { - globalPriv.Delete(itemGlobalPriv{username: u}) + for u := range userList { + itm, ok := diff.globalPriv.Get(itemGlobalPriv{username: u}) + if !ok { + globalPriv.Delete(itemGlobalPriv{username: u}) + } else { + slices.SortFunc(itm.data, compareGlobalPrivRecord) + globalPriv.ReplaceOrInsert(itm) + } } - diff.globalPriv.Ascend(func(itm itemGlobalPriv) bool { - slices.SortFunc(itm.data, compareGlobalPrivRecord) - globalPriv.ReplaceOrInsert(itm) - return true - }) ret.globalPriv.BTreeG = globalPriv ret.roleGraph = diff.roleGraph @@ -690,9 +681,9 @@ func noSuchTable(err error) bool { } // LoadRoleGraph loads the mysql.role_edges table from database. -func (p *MySQLPrivilege) LoadRoleGraph(ctx sqlexec.RestrictedSQLExecutor) error { - p.roleGraph = make(map[string]roleGraphEdgesTable) - err := p.loadTable(ctx, sqlLoadRoleGraph, p.decodeRoleEdgesTable) +func (p *MySQLPrivilege) LoadRoleGraph(exec sqlexec.SQLExecutor) error { + p.roleGraph = make(map[auth.RoleIdentity]roleGraphEdgesTable) + err := loadTable(exec, sqlLoadRoleGraph, p.decodeRoleEdgesTable) if err != nil { return errors.Trace(err) } @@ -700,8 +691,8 @@ func (p *MySQLPrivilege) LoadRoleGraph(ctx sqlexec.RestrictedSQLExecutor) error } // LoadUserTable loads the mysql.user table from database. -func (p *MySQLPrivilege) LoadUserTable(ctx sqlexec.RestrictedSQLExecutor) error { - err := p.loadTable(ctx, sqlLoadUserTable, p.decodeUserTableRow) +func (p *MySQLPrivilege) LoadUserTable(exec sqlexec.SQLExecutor) error { + err := loadTable(exec, sqlLoadUserTable, p.decodeUserTableRow(nil)) if err != nil { return errors.Trace(err) } @@ -825,24 +816,24 @@ func (p *MySQLPrivilege) SortUserTable() { } // LoadGlobalPrivTable loads the mysql.global_priv table from database. -func (p *MySQLPrivilege) LoadGlobalPrivTable(ctx sqlexec.RestrictedSQLExecutor) error { - if err := p.loadTable(ctx, sqlLoadGlobalPrivTable, p.decodeGlobalPrivTableRow); err != nil { +func (p *MySQLPrivilege) LoadGlobalPrivTable(exec sqlexec.SQLExecutor) error { + if err := loadTable(exec, sqlLoadGlobalPrivTable, p.decodeGlobalPrivTableRow(nil)); err != nil { return errors.Trace(err) } return nil } // LoadGlobalGrantsTable loads the mysql.global_priv table from database. -func (p *MySQLPrivilege) LoadGlobalGrantsTable(ctx sqlexec.RestrictedSQLExecutor) error { - if err := p.loadTable(ctx, sqlLoadGlobalGrantsTable, p.decodeGlobalGrantsTableRow); err != nil { +func (p *MySQLPrivilege) LoadGlobalGrantsTable(exec sqlexec.SQLExecutor) error { + if err := loadTable(exec, sqlLoadGlobalGrantsTable, p.decodeGlobalGrantsTableRow(nil)); err != nil { return errors.Trace(err) } return nil } // LoadDBTable loads the mysql.db table from database. -func (p *MySQLPrivilege) LoadDBTable(ctx sqlexec.RestrictedSQLExecutor) error { - err := p.loadTable(ctx, sqlLoadDBTable, p.decodeDBTableRow) +func (p *MySQLPrivilege) LoadDBTable(exec sqlexec.SQLExecutor) error { + err := loadTable(exec, sqlLoadDBTable, p.decodeDBTableRow(nil)) if err != nil { return err } @@ -877,8 +868,8 @@ func compareTablesPrivRecord(x, y tablesPrivRecord) int { } // LoadTablesPrivTable loads the mysql.tables_priv table from database. -func (p *MySQLPrivilege) LoadTablesPrivTable(ctx sqlexec.RestrictedSQLExecutor) error { - err := p.loadTable(ctx, sqlLoadTablePrivTable, p.decodeTablesPrivTableRow) +func (p *MySQLPrivilege) LoadTablesPrivTable(exec sqlexec.SQLExecutor) error { + err := loadTable(exec, sqlLoadTablePrivTable, p.decodeTablesPrivTableRow(nil)) if err != nil { return err } @@ -886,49 +877,67 @@ func (p *MySQLPrivilege) LoadTablesPrivTable(ctx sqlexec.RestrictedSQLExecutor) } // LoadColumnsPrivTable loads the mysql.columns_priv table from database. -func (p *MySQLPrivilege) LoadColumnsPrivTable(ctx sqlexec.RestrictedSQLExecutor) error { - return p.loadTable(ctx, sqlLoadColumnsPrivTable, p.decodeColumnsPrivTableRow) +func (p *MySQLPrivilege) LoadColumnsPrivTable(exec sqlexec.SQLExecutor) error { + return loadTable(exec, sqlLoadColumnsPrivTable, p.decodeColumnsPrivTableRow(nil)) } // LoadDefaultRoles loads the mysql.columns_priv table from database. -func (p *MySQLPrivilege) LoadDefaultRoles(ctx sqlexec.RestrictedSQLExecutor) error { - return p.loadTable(ctx, sqlLoadDefaultRoles, p.decodeDefaultRoleTableRow) +func (p *MySQLPrivilege) LoadDefaultRoles(exec sqlexec.SQLExecutor) error { + return loadTable(exec, sqlLoadDefaultRoles, p.decodeDefaultRoleTableRow(nil)) } -func addUserFilterCondition(sql string, userList []string) string { - if len(userList) == 0 { +func addUserFilterCondition(sql string, userList map[string]struct{}) string { + if len(userList) == 0 || len(userList) > 1024 { return sql } var b strings.Builder b.WriteString(sql) b.WriteString(" WHERE ") - for i, user := range userList { - if i > 0 { + first := true + for user := range userList { + if !first { b.WriteString(" OR ") + } else { + first = false } fmt.Fprintf(&b, "USER = '%s'", sqlescape.EscapeString(user)) } return b.String() } -func (p *MySQLPrivilege) loadTable(sctx sqlexec.RestrictedSQLExecutor, sql string, - decodeTableRow func(chunk.Row, []*resolve.ResultField) error, userList ...string) error { +func loadTable(exec sqlexec.SQLExecutor, sql string, + decodeTableRow func(chunk.Row, []*resolve.ResultField) error) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnPrivilege) - sql = addUserFilterCondition(sql, userList) - rows, fs, err := sctx.ExecRestrictedSQL(ctx, nil, sql) + // Do not use sctx.ExecRestrictedSQL() here deliberately. + // The result set can be extremely large, so this streaming API is important to + // reduce memory cost. + rs, err := exec.ExecuteInternal(ctx, sql) if err != nil { return errors.Trace(err) } - for _, row := range rows { - // NOTE: decodeTableRow decodes data from a chunk Row, that is a shallow copy. - // The result will reference memory in the chunk, so the chunk must not be reused - // here, otherwise some werid bug will happen! - err = decodeTableRow(row, fs) + defer terror.Call(rs.Close) + fs := rs.Fields() + req := rs.NewChunk(nil) + for { + err = rs.Next(ctx, req) if err != nil { return errors.Trace(err) } + if req.NumRows() == 0 { + return nil + } + it := chunk.NewIterator4Chunk(req) + for row := it.Begin(); row != it.End(); row = it.Next() { + err = decodeTableRow(row, fs) + if err != nil { + return errors.Trace(err) + } + } + // NOTE: decodeTableRow decodes data from a chunk Row, that is a shallow copy. + // The result will reference memory in the chunk, so the chunk must not be reused + // here, otherwise some werid bug will happen! + req = chunk.Renew(req, 1024) } - return nil } // parseHostIPNet parses an IPv4 address and its subnet mask (e.g. `127.0.0.0/255.255.255.0`), @@ -964,219 +973,265 @@ func parseHostIPNet(s string) *net.IPNet { func (record *baseRecord) assignUserOrHost(row chunk.Row, i int, f *resolve.ResultField) { switch f.ColumnAsName.L { case "user": - record.User = row.GetString(i) + record.User = strings.Clone(row.GetString(i)) case "host": - record.Host = row.GetString(i) + record.Host = strings.Clone(row.GetString(i)) record.patChars, record.patTypes = stringutil.CompilePatternBinary(record.Host, '\\') record.hostIPNet = parseHostIPNet(record.Host) } } -func (p *MySQLPrivilege) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value UserRecord - for i, f := range fs { - switch { - case f.ColumnAsName.L == "authentication_string": - value.AuthenticationString = row.GetString(i) - case f.ColumnAsName.L == "account_locked": - if row.GetEnum(i).String() == "Y" { - value.AccountLocked = true +func (p *MySQLPrivilege) decodeUserTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value UserRecord + defaultAuthPlugin := "" + if p.globalVars != nil { + val, err := p.globalVars.GetGlobalSysVar(vardef.DefaultAuthPlugin) + if err == nil { + defaultAuthPlugin = val } - case f.ColumnAsName.L == "plugin": - if row.GetString(i) != "" { - value.AuthPlugin = row.GetString(i) - } else { - value.AuthPlugin = mysql.AuthNativePassword - } - case f.ColumnAsName.L == "token_issuer": - value.AuthTokenIssuer = row.GetString(i) - case f.ColumnAsName.L == "user_attributes": - if row.IsNull(i) { - continue - } - bj := row.GetJSON(i) - pathExpr, err := types.ParseJSONPathExpr("$.metadata.email") - if err != nil { - return err - } - if emailBJ, found := bj.Extract([]types.JSONPathExpression{pathExpr}); found { - email, err := emailBJ.Unquote() + } + if defaultAuthPlugin == "" { + defaultAuthPlugin = mysql.AuthNativePassword + } + for i, f := range fs { + switch { + case f.ColumnAsName.L == "authentication_string": + value.AuthenticationString = strings.Clone(row.GetString(i)) + case f.ColumnAsName.L == "account_locked": + if row.GetEnum(i).String() == "Y" { + value.AccountLocked = true + } + case f.ColumnAsName.L == "plugin": + if row.GetString(i) != "" { + value.AuthPlugin = strings.Clone(row.GetString(i)) + } else { + value.AuthPlugin = defaultAuthPlugin + } + case f.ColumnAsName.L == "token_issuer": + value.AuthTokenIssuer = strings.Clone(row.GetString(i)) + case f.ColumnAsName.L == "user_attributes": + if row.IsNull(i) { + continue + } + bj := row.GetJSON(i) + pathExpr, err := types.ParseJSONPathExpr("$.metadata.email") if err != nil { return err } - value.Email = email - } - pathExpr, err = types.ParseJSONPathExpr("$.resource_group") - if err != nil { - return err - } - if resourceGroup, found := bj.Extract([]types.JSONPathExpression{pathExpr}); found { - resourceGroup, err := resourceGroup.Unquote() + if emailBJ, found := bj.Extract([]types.JSONPathExpression{pathExpr}); found { + email, err := emailBJ.Unquote() + if err != nil { + return err + } + value.Email = strings.Clone(email) + } + pathExpr, err = types.ParseJSONPathExpr("$.resource_group") if err != nil { return err } - value.ResourceGroup = resourceGroup - } - passwordLocking := PasswordLocking{} - if err := passwordLocking.ParseJSON(bj); err != nil { - return err - } - value.FailedLoginAttempts = passwordLocking.FailedLoginAttempts - value.PasswordLockTimeDays = passwordLocking.PasswordLockTimeDays - value.FailedLoginCount = passwordLocking.FailedLoginCount - value.AutoLockedLastChanged = passwordLocking.AutoLockedLastChanged - value.AutoAccountLocked = passwordLocking.AutoAccountLocked - case f.ColumnAsName.L == "password_expired": - if row.GetEnum(i).String() == "Y" { - value.PasswordExpired = true - } - case f.ColumnAsName.L == "password_last_changed": - t := row.GetTime(i) - gotime, err := t.GoTime(time.Local) - if err != nil { - return err - } - value.PasswordLastChanged = gotime - case f.ColumnAsName.L == "password_lifetime": - if row.IsNull(i) { - value.PasswordLifeTime = -1 - continue - } - value.PasswordLifeTime = row.GetInt64(i) - case f.Column.GetType() == mysql.TypeEnum: - if row.GetEnum(i).String() != "Y" { - continue - } - priv, ok := mysql.Col2PrivType[f.ColumnAsName.O] - if !ok { - return errInvalidPrivilegeType.GenWithStack(f.ColumnAsName.O) + if resourceGroup, found := bj.Extract([]types.JSONPathExpression{pathExpr}); found { + resourceGroup, err := resourceGroup.Unquote() + if err != nil { + return err + } + value.ResourceGroup = strings.Clone(resourceGroup) + } + passwordLocking := PasswordLocking{} + if err := passwordLocking.ParseJSON(bj); err != nil { + return err + } + value.FailedLoginAttempts = passwordLocking.FailedLoginAttempts + value.PasswordLockTimeDays = passwordLocking.PasswordLockTimeDays + value.FailedLoginCount = passwordLocking.FailedLoginCount + value.AutoLockedLastChanged = passwordLocking.AutoLockedLastChanged + value.AutoAccountLocked = passwordLocking.AutoAccountLocked + case f.ColumnAsName.L == "password_expired": + if row.GetEnum(i).String() == "Y" { + value.PasswordExpired = true + } + case f.ColumnAsName.L == "password_last_changed": + t := row.GetTime(i) + gotime, err := t.GoTime(time.Local) + if err != nil { + return err + } + value.PasswordLastChanged = gotime + case f.ColumnAsName.L == "password_lifetime": + if row.IsNull(i) { + value.PasswordLifeTime = -1 + continue + } + value.PasswordLifeTime = row.GetInt64(i) + case f.ColumnAsName.L == "max_user_connections": + value.MaxUserConnections = row.GetInt64(i) + case f.Column.GetType() == mysql.TypeEnum: + if row.GetEnum(i).String() != "Y" { + continue + } + priv, ok := mysql.Col2PrivType[f.ColumnAsName.O] + if !ok { + return errInvalidPrivilegeType.GenWithStack(f.ColumnAsName.O) + } + value.Privileges |= priv + default: + value.assignUserOrHost(row, i, f) } - value.Privileges |= priv - default: - value.assignUserOrHost(row, i, f) } + old, ok := p.user.Get(itemUser{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.user.ReplaceOrInsert(old) + return nil } - old, ok := p.user.Get(itemUser{username: value.User}) - if !ok { - old.username = value.User - } - old.data = append(old.data, value) - p.user.ReplaceOrInsert(old) - return nil } -func (p *MySQLPrivilege) decodeGlobalPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value globalPrivRecord - for i, f := range fs { - if f.ColumnAsName.L == "priv" { - privData := row.GetString(i) - if len(privData) > 0 { - var privValue GlobalPrivValue - err := json.Unmarshal(hack.Slice(privData), &privValue) - if err != nil { - logutil.BgLogger().Error("one user global priv data is broken, forbidden login until data be fixed", - zap.String("user", value.User), zap.String("host", value.Host)) - value.Broken = true - } else { - value.Priv.SSLType = privValue.SSLType - value.Priv.SSLCipher = privValue.SSLCipher - value.Priv.X509Issuer = privValue.X509Issuer - value.Priv.X509Subject = privValue.X509Subject - value.Priv.SAN = privValue.SAN - if len(value.Priv.SAN) > 0 { - value.Priv.SANs, err = util.ParseAndCheckSAN(value.Priv.SAN) - if err != nil { - value.Broken = true +func (p *MySQLPrivilege) decodeGlobalPrivTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value globalPrivRecord + for i, f := range fs { + if f.ColumnAsName.L == "priv" { + privData := row.GetString(i) + if len(privData) > 0 { + var privValue GlobalPrivValue + err := json.Unmarshal(hack.Slice(privData), &privValue) + if err != nil { + logutil.BgLogger().Error("one user global priv data is broken, forbidden login until data be fixed", + zap.String("user", value.User), zap.String("host", value.Host)) + value.Broken = true + } else { + value.Priv.SSLType = privValue.SSLType + value.Priv.SSLCipher = strings.Clone(privValue.SSLCipher) + value.Priv.X509Issuer = strings.Clone(privValue.X509Issuer) + value.Priv.X509Subject = strings.Clone(privValue.X509Subject) + value.Priv.SAN = strings.Clone(privValue.SAN) + if len(value.Priv.SAN) > 0 { + value.Priv.SANs, err = util.ParseAndCheckSAN(value.Priv.SAN) + if err != nil { + value.Broken = true + } } } } + } else { + value.assignUserOrHost(row, i, f) } - } else { - value.assignUserOrHost(row, i, f) } + if userList != nil { + if _, ok := userList[value.User]; !ok { + return nil + } + } + + old, ok := p.globalPriv.Get(itemGlobalPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.globalPriv.ReplaceOrInsert(old) + return nil } - old, ok := p.globalPriv.Get(itemGlobalPriv{username: value.User}) - if !ok { - old.username = value.User - } - old.data = append(old.data, value) - p.globalPriv.ReplaceOrInsert(old) - return nil } -func (p *MySQLPrivilege) decodeGlobalGrantsTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value dynamicPrivRecord - for i, f := range fs { - switch f.ColumnAsName.L { - case "priv": - value.PrivilegeName = strings.ToUpper(row.GetString(i)) - case "with_grant_option": - value.GrantOption = row.GetEnum(i).String() == "Y" - default: - value.assignUserOrHost(row, i, f) +func (p *MySQLPrivilege) decodeGlobalGrantsTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value dynamicPrivRecord + for i, f := range fs { + switch f.ColumnAsName.L { + case "priv": + value.PrivilegeName = strings.ToUpper(row.GetString(i)) + case "with_grant_option": + value.GrantOption = row.GetEnum(i).String() == "Y" + default: + value.assignUserOrHost(row, i, f) + } } + if userList != nil { + if _, ok := userList[value.User]; !ok { + return nil + } + } + + old, ok := p.dynamicPriv.Get(itemDynamicPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.dynamicPriv.ReplaceOrInsert(old) + return nil } - old, ok := p.dynamicPriv.Get(itemDynamicPriv{username: value.User}) - if !ok { - old.username = value.User - } - old.data = append(old.data, value) - p.dynamicPriv.ReplaceOrInsert(old) - return nil } -func (p *MySQLPrivilege) decodeDBTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value dbRecord - for i, f := range fs { - switch { - case f.ColumnAsName.L == "db": - value.DB = row.GetString(i) - value.dbPatChars, value.dbPatTypes = stringutil.CompilePatternBinary(strings.ToUpper(value.DB), '\\') - case f.Column.GetType() == mysql.TypeEnum: - if row.GetEnum(i).String() != "Y" { - continue +func (p *MySQLPrivilege) decodeDBTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value dbRecord + for i, f := range fs { + switch { + case f.ColumnAsName.L == "db": + value.DB = row.GetString(i) + value.dbPatChars, value.dbPatTypes = stringutil.CompilePatternBinary(strings.ToUpper(value.DB), '\\') + case f.Column.GetType() == mysql.TypeEnum: + if row.GetEnum(i).String() != "Y" { + continue + } + priv, ok := mysql.Col2PrivType[f.ColumnAsName.O] + if !ok { + return errInvalidPrivilegeType.GenWithStack("Unknown Privilege Type!") + } + value.Privileges |= priv + default: + value.assignUserOrHost(row, i, f) } - priv, ok := mysql.Col2PrivType[f.ColumnAsName.O] - if !ok { - return errInvalidPrivilegeType.GenWithStack("Unknown Privilege Type!") + } + if userList != nil { + if _, ok := userList[value.User]; !ok { + return nil } - value.Privileges |= priv - default: - value.assignUserOrHost(row, i, f) } + + old, ok := p.db.Get(itemDB{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.db.ReplaceOrInsert(old) + return nil } - old, ok := p.db.Get(itemDB{username: value.User}) - if !ok { - old.username = value.User - } - old.data = append(old.data, value) - p.db.ReplaceOrInsert(old) - return nil } -func (p *MySQLPrivilege) decodeTablesPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value tablesPrivRecord - for i, f := range fs { - switch f.ColumnAsName.L { - case "db": - value.DB = row.GetString(i) - case "table_name": - value.TableName = row.GetString(i) - case "table_priv": - value.TablePriv = decodeSetToPrivilege(row.GetSet(i)) - case "column_priv": - value.ColumnPriv = decodeSetToPrivilege(row.GetSet(i)) - default: - value.assignUserOrHost(row, i, f) - } - } - old, ok := p.tablesPriv.Get(itemTablesPriv{username: value.User}) - if !ok { - old.username = value.User +func (p *MySQLPrivilege) decodeTablesPrivTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value tablesPrivRecord + for i, f := range fs { + switch f.ColumnAsName.L { + case "db": + value.DB = row.GetString(i) + case "table_name": + value.TableName = row.GetString(i) + case "table_priv": + value.TablePriv = decodeSetToPrivilege(row.GetSet(i)) + case "column_priv": + value.ColumnPriv = decodeSetToPrivilege(row.GetSet(i)) + default: + value.assignUserOrHost(row, i, f) + } + } + if userList != nil { + if _, ok := userList[value.User]; !ok { + return nil + } + } + + old, ok := p.tablesPriv.Get(itemTablesPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.tablesPriv.ReplaceOrInsert(old) + return nil } - old.data = append(old.data, value) - p.tablesPriv.ReplaceOrInsert(old) - return nil } func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.ResultField) error { @@ -1204,56 +1259,72 @@ func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.Resul return nil } -func (p *MySQLPrivilege) decodeDefaultRoleTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value defaultRoleRecord - for i, f := range fs { - switch f.ColumnAsName.L { - case "default_role_host": - value.DefaultRoleHost = row.GetString(i) - case "default_role_user": - value.DefaultRoleUser = row.GetString(i) - default: - value.assignUserOrHost(row, i, f) +func (p *MySQLPrivilege) decodeDefaultRoleTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value defaultRoleRecord + for i, f := range fs { + switch f.ColumnAsName.L { + case "default_role_host": + value.DefaultRoleHost = row.GetString(i) + case "default_role_user": + value.DefaultRoleUser = row.GetString(i) + default: + value.assignUserOrHost(row, i, f) + } } + if userList != nil { + if _, ok := userList[value.User]; !ok { + return nil + } + } + + old, ok := p.defaultRoles.Get(itemDefaultRole{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.defaultRoles.ReplaceOrInsert(old) + return nil } - old, ok := p.defaultRoles.Get(itemDefaultRole{username: value.User}) - if !ok { - old.username = value.User - } - old.data = append(old.data, value) - p.defaultRoles.ReplaceOrInsert(old) - return nil } -func (p *MySQLPrivilege) decodeColumnsPrivTableRow(row chunk.Row, fs []*resolve.ResultField) error { - var value columnsPrivRecord - for i, f := range fs { - switch f.ColumnAsName.L { - case "db": - value.DB = row.GetString(i) - case "table_name": - value.TableName = row.GetString(i) - case "column_name": - value.ColumnName = row.GetString(i) - case "timestamp": - var err error - value.Timestamp, err = row.GetTime(i).GoTime(time.Local) - if err != nil { - return errors.Trace(err) +func (p *MySQLPrivilege) decodeColumnsPrivTableRow(userList map[string]struct{}) func(chunk.Row, []*resolve.ResultField) error { + return func(row chunk.Row, fs []*resolve.ResultField) error { + var value columnsPrivRecord + for i, f := range fs { + switch f.ColumnAsName.L { + case "db": + value.DB = row.GetString(i) + case "table_name": + value.TableName = row.GetString(i) + case "column_name": + value.ColumnName = row.GetString(i) + case "timestamp": + var err error + value.Timestamp, err = row.GetTime(i).GoTime(time.Local) + if err != nil { + return errors.Trace(err) + } + case "column_priv": + value.ColumnPriv = decodeSetToPrivilege(row.GetSet(i)) + default: + value.assignUserOrHost(row, i, f) } - case "column_priv": - value.ColumnPriv = decodeSetToPrivilege(row.GetSet(i)) - default: - value.assignUserOrHost(row, i, f) } + if userList != nil { + if _, ok := userList[value.User]; !ok { + return nil + } + } + + old, ok := p.columnsPriv.Get(itemColumnsPriv{username: value.User}) + if !ok { + old.username = value.User + } + old.data = append(old.data, value) + p.columnsPriv.ReplaceOrInsert(old) + return nil } - old, ok := p.columnsPriv.Get(itemColumnsPriv{username: value.User}) - if !ok { - old.username = value.User - } - old.data = append(old.data, value) - p.columnsPriv.ReplaceOrInsert(old) - return nil } func decodeSetToPrivilege(s types.Set) mysql.PrivilegeType { @@ -1326,7 +1397,7 @@ func patternMatch(str string, patChars, patTypes []byte) bool { // matchIdentity finds an identity to match a user + host // using the correct rules according to MySQL. -func (p *MySQLPrivilege) matchIdentity(sctx sqlexec.RestrictedSQLExecutor, user, host string, skipNameResolve bool) *UserRecord { +func (p *MySQLPrivilege) matchIdentity(user, host string, skipNameResolve bool) *UserRecord { item, ok := p.user.Get(itemUser{username: user}) if !ok { return nil @@ -2008,7 +2079,7 @@ func (p *MySQLPrivilege) getAllRoles(user, host string) []*auth.RoleIdentity { // Handle wraps MySQLPrivilege providing thread safe access. type Handle struct { - sctx sqlexec.RestrictedSQLExecutor + sctx util.SessionPool priv atomic.Pointer[MySQLPrivilege] // Only load the active user's data to save memory // username => struct{} @@ -2018,7 +2089,7 @@ type Handle struct { } // NewHandle returns a Handle. -func NewHandle(sctx sqlexec.RestrictedSQLExecutor, globalVars variable.GlobalVarAccessor) *Handle { +func NewHandle(sctx util.SessionPool, globalVars variable.GlobalVarAccessor) *Handle { priv := newMySQLPrivilege() ret := &Handle{} ret.sctx = sctx @@ -2042,18 +2113,10 @@ func (h *Handle) ensureActiveUser(ctx context.Context, user string) error { if exist { return nil } - data := newMySQLPrivilege() - data.globalVars = h.globalVars - userList, err := data.loadSomeUsers(h.sctx, user) - if err != nil { - return errors.Trace(err) - } - - h.merge(data, userList) - return nil + return h.updateUsers([]string{user}) } -func (h *Handle) merge(data *MySQLPrivilege, userList []string) { +func (h *Handle) merge(data *MySQLPrivilege, userList map[string]struct{}) { for { old := h.Get() swapped := h.priv.CompareAndSwap(old, old.merge(data, userList)) @@ -2061,7 +2124,7 @@ func (h *Handle) merge(data *MySQLPrivilege, userList []string) { break } } - for _, user := range userList { + for user := range userList { h.activeUsers.Store(user, struct{}{}) } } @@ -2073,9 +2136,15 @@ func (h *Handle) Get() *MySQLPrivilege { // UpdateAll loads all the users' privilege info from kv storage. func (h *Handle) UpdateAll() error { - logutil.BgLogger().Warn("update all called") priv := newMySQLPrivilege() - err := priv.LoadAll(h.sctx) + res, err := h.sctx.Get() + if err != nil { + return errors.Trace(err) + } + defer h.sctx.Put(res) + exec := res.(sqlexec.SQLExecutor) + + err = priv.LoadAll(exec) if err != nil { return errors.Trace(err) } @@ -2092,19 +2161,7 @@ func (h *Handle) UpdateAllActive() error { userList = append(userList, key.(string)) return true }) - if len(userList) > 1024 { - logutil.BgLogger().Warn("active user count > 1024, revert to update all", zap.Int("len", len(userList))) - return h.UpdateAll() - } - - priv := newMySQLPrivilege() - priv.globalVars = h.globalVars - userList, err := priv.loadSomeUsers(h.sctx, userList...) - if err != nil { - return err - } - h.merge(priv, userList) - return nil + return h.updateUsers(userList) } // Update loads the privilege info from kv storage for the list of users. @@ -2124,12 +2181,32 @@ func (h *Handle) Update(userList []string) error { return nil } - priv := newMySQLPrivilege() - priv.globalVars = h.globalVars - userList, err := priv.loadSomeUsers(h.sctx, userList...) + return h.updateUsers(userList) +} + +func (h *Handle) updateUsers(userList []string) error { + res, err := h.sctx.Get() + if err != nil { + return errors.Trace(err) + } + defer h.sctx.Put(res) + exec := res.(sqlexec.SQLExecutor) + + p := newMySQLPrivilege() + p.globalVars = h.globalVars + // Load the full role edge table first. + p.roleGraph = make(map[auth.RoleIdentity]roleGraphEdgesTable) + err = loadTable(exec, sqlLoadRoleGraph, p.decodeRoleEdgesTable) + if err != nil { + return errors.Trace(err) + } + + // Including the user and also their roles + userAndRoles := findUserAndAllRoles(userList, p.roleGraph) + err = p.loadSomeUsers(exec, userAndRoles) if err != nil { return err } - h.merge(priv, userList) + h.merge(p, userAndRoles) return nil } diff --git a/pkg/privilege/privileges/cache_test.go b/pkg/privilege/privileges/cache_test.go index c1504244d1406..a2181b59807c4 100644 --- a/pkg/privilege/privileges/cache_test.go +++ b/pkg/privilege/privileges/cache_test.go @@ -36,7 +36,7 @@ func TestLoadUserTable(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.Len(t, p.User(), 0) // Host | User | authentication_string | Select_priv | Insert_priv | Update_priv | Delete_priv | Create_priv | Drop_priv | Process_priv | Grant_priv | References_priv | Alter_priv | Show_db_priv | Super_priv | Execute_priv | Index_priv | Create_user_priv | Trigger_priv @@ -49,7 +49,7 @@ func TestLoadUserTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.user (Host, User, password_expired, password_last_changed) VALUES ("%", "root3", "N", "2022-10-10 12:00:00")`) p = privileges.NewMySQLPrivilege() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) // require.Len(t, p.User(), len(p.UserMap)) user := p.User() @@ -72,7 +72,7 @@ func TestLoadUserTable(t *testing.T) { p = privileges.NewMySQLPrivilege() p.SetGlobalVarsAccessor(se.GetSessionVars().GlobalVarsAccessor) require.NoError(t, se.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.DefaultAuthPlugin, plugin)) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.Equal(t, plugin, p.User()[0].AuthPlugin) } } @@ -89,7 +89,7 @@ func TestLoadGlobalPrivTable(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadGlobalPrivTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadGlobalPrivTable(se.GetSQLExecutor())) val := p.GlobalPriv("tu")[0] require.Equal(t, `%`, val.Host) require.Equal(t, `tu`, val.User) @@ -114,7 +114,7 @@ func TestLoadDBTable(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadDBTable(se.GetSQLExecutor())) // require.Len(t, p.DB(), len(p.DBMap)) require.Equal(t, mysql.SelectPriv|mysql.InsertPriv|mysql.UpdatePriv|mysql.DeletePriv|mysql.CreatePriv, p.DB()[0].Privileges) @@ -132,7 +132,7 @@ func TestLoadTablesPrivTable(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadTablesPrivTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadTablesPrivTable(se.GetSQLExecutor())) tablesPriv := p.TablesPriv() // require.Len(t, tablesPriv, len(p.TablesPrivMap)) @@ -156,7 +156,7 @@ func TestLoadColumnsPrivTable(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadColumnsPrivTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadColumnsPrivTable(se.GetSQLExecutor())) columnsPriv := p.ColumnsPriv() require.Equal(t, `%`, columnsPriv[0].Host) require.Equal(t, "db", columnsPriv[0].DB) @@ -178,7 +178,7 @@ func TestLoadDefaultRoleTable(t *testing.T) { tk.MustExec(`INSERT INTO mysql.default_roles VALUES ("%", "test_default_roles", "localhost", "r_2")`) p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadDefaultRoles(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadDefaultRoles(se.GetSQLExecutor())) require.Equal(t, `%`, p.DefaultRoles()[0].Host) require.Equal(t, "test_default_roles", p.DefaultRoles()[0].User) require.Equal(t, "localhost", p.DefaultRoles()[0].DefaultRoleHost) @@ -197,7 +197,7 @@ func TestPatternMatch(t *testing.T) { tk.MustExec(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("10.0.%", "root", "Y", "Y")`) p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "root", "10.0.1", "test", "", "", mysql.SelectPriv)) require.True(t, p.RequestVerification(activeRoles, "root", "10.0.1.118", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "localhost", "test", "", "", mysql.SelectPriv)) @@ -209,7 +209,7 @@ func TestPatternMatch(t *testing.T) { tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("", "root", "Y", "N")`) p = privileges.NewMySQLPrivilege() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "root", "", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "notnull", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "", "test", "", "", mysql.ShutdownPriv)) @@ -218,7 +218,7 @@ func TestPatternMatch(t *testing.T) { tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec("TRUNCATE TABLE mysql.db") tk.MustExec(`INSERT INTO mysql.db (user,host,db,select_priv) values ('genius', '%', 'te%', 'Y')`) - require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadDBTable(se.GetSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "genius", "127.0.0.1", "test", "", "", mysql.SelectPriv)) } @@ -234,7 +234,7 @@ func TestHostMatch(t *testing.T) { tk.MustExec(`INSERT INTO mysql.user (HOST, USER, authentication_string, Select_priv, Shutdown_priv) VALUES ("172.0.0.0/255.0.0.0", "root", "", "Y", "Y")`) p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.True(t, p.RequestVerification(activeRoles, "root", "172.0.0.1", "test", "", "", mysql.SelectPriv)) require.True(t, p.RequestVerification(activeRoles, "root", "172.1.1.1", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "localhost", "test", "", "", mysql.SelectPriv)) @@ -260,7 +260,7 @@ func TestHostMatch(t *testing.T) { tk.MustExec(sql) p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.False(t, p.RequestVerification(activeRoles, "root", "127.0.0.1", "test", "", "", mysql.SelectPriv), fmt.Sprintf("test case: %s", IPMask)) require.False(t, p.RequestVerification(activeRoles, "root", "127.0.0.0", "test", "", "", mysql.SelectPriv), fmt.Sprintf("test case: %s", IPMask)) require.False(t, p.RequestVerification(activeRoles, "root", "localhost", "test", "", "", mysql.ShutdownPriv), fmt.Sprintf("test case: %s", IPMask)) @@ -269,7 +269,7 @@ func TestHostMatch(t *testing.T) { // Netmask notation cannot be used for IPv6 addresses. tk.MustExec(`INSERT INTO mysql.user (HOST, USER, Select_priv, Shutdown_priv) VALUES ("2001:db8::/ffff:ffff::", "root", "Y", "Y")`) p = privileges.NewMySQLPrivilege() - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.False(t, p.RequestVerification(activeRoles, "root", "2001:db8::1234", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "2001:db8::", "test", "", "", mysql.SelectPriv)) require.False(t, p.RequestVerification(activeRoles, "root", "localhost", "test", "", "", mysql.ShutdownPriv)) @@ -286,7 +286,7 @@ func TestCaseInsensitive(t *testing.T) { tk.MustExec(`INSERT INTO mysql.db VALUES ("127.0.0.1", "TCTrain", "genius", "Y", "Y", "Y", "Y", "Y", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N", "N")`) p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadDBTable(se.GetSQLExecutor())) // DB and Table names are case-insensitive in MySQL. require.True(t, p.RequestVerification(activeRoles, "genius", "127.0.0.1", "TCTrain", "TCTrainOrder", "", mysql.SelectPriv)) require.True(t, p.RequestVerification(activeRoles, "genius", "127.0.0.1", "TCTRAIN", "TCTRAINORDER", "", mysql.SelectPriv)) @@ -302,7 +302,7 @@ func TestLoadRoleGraph(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadDBTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadDBTable(se.GetSQLExecutor())) require.Len(t, p.User(), 0) tk.MustExec(`INSERT INTO mysql.role_edges (FROM_HOST, FROM_USER, TO_HOST, TO_USER) VALUES ("%", "r_1", "%", "user2")`) @@ -311,7 +311,7 @@ func TestLoadRoleGraph(t *testing.T) { tk.MustExec(`INSERT INTO mysql.role_edges (FROM_HOST, FROM_USER, TO_HOST, TO_USER) VALUES ("%", "r_4", "%", "root")`) p = privileges.NewMySQLPrivilege() - require.NoError(t, p.LoadRoleGraph(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadRoleGraph(se.GetSQLExecutor())) graph := p.RoleGraph() require.True(t, graph["root@%"].Find("r_2", "%")) require.True(t, graph["root@%"].Find("r_4", "%")) @@ -335,7 +335,7 @@ func TestRoleGraphBFS(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadRoleGraph(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadRoleGraph(se.GetSQLExecutor())) activeRoles := make([]*auth.RoleIdentity, 0) ret := p.FindAllRole(activeRoles) @@ -369,7 +369,7 @@ func TestFindAllUserEffectiveRoles(t *testing.T) { p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadAll(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadAll(se.GetSQLExecutor())) ret := p.FindAllUserEffectiveRoles("u1", "%", []*auth.RoleIdentity{ {Username: "r_1", Hostname: "%"}, {Username: "r_2", Hostname: "%"}, @@ -381,7 +381,7 @@ func TestFindAllUserEffectiveRoles(t *testing.T) { require.Equal(t, "r_4", ret[3].Username) tk.MustExec(`REVOKE r_2 from u1`) - require.NoError(t, p.LoadAll(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadAll(se.GetSQLExecutor())) ret = p.FindAllUserEffectiveRoles("u1", "%", []*auth.RoleIdentity{ {Username: "r_1", Hostname: "%"}, {Username: "r_2", Hostname: "%"}, @@ -465,58 +465,58 @@ func TestDBIsVisible(t *testing.T) { tk.MustExec("create database visdb") p := privileges.NewMySQLPrivilege() se := tk.Session() - require.NoError(t, p.LoadAll(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadAll(se.GetSQLExecutor())) tk.MustExec(`INSERT INTO mysql.user (Host, User, Create_role_priv, Super_priv) VALUES ("%", "testvisdb", "Y", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible := p.DBIsVisible("testvisdb", "%", "visdb") require.False(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Select_priv) VALUES ("%", "testvisdb2", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb2", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Create_priv) VALUES ("%", "testvisdb3", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb3", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Insert_priv) VALUES ("%", "testvisdb4", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb4", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Update_priv) VALUES ("%", "testvisdb5", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb5", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Create_view_priv) VALUES ("%", "testvisdb6", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb6", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Trigger_priv) VALUES ("%", "testvisdb7", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb7", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, References_priv) VALUES ("%", "testvisdb8", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb8", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") tk.MustExec(`INSERT INTO mysql.user (Host, User, Execute_priv) VALUES ("%", "testvisdb9", "Y")`) - require.NoError(t, p.LoadUserTable(se.GetRestrictedSQLExecutor())) + require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) isVisible = p.DBIsVisible("testvisdb9", "%", "visdb") require.True(t, isVisible) tk.MustExec("TRUNCATE TABLE mysql.user") diff --git a/pkg/privilege/privileges/privileges.go b/pkg/privilege/privileges/privileges.go index b5aad336d080f..8e30fd4748df6 100644 --- a/pkg/privilege/privileges/privileges.go +++ b/pkg/privilege/privileges/privileges.go @@ -393,7 +393,7 @@ func (p *UserPrivileges) MatchIdentity(user, host string, skipNameResolve bool) zap.String("user", user)) } mysqlPriv := p.Handle.Get() - record := mysqlPriv.matchIdentity(p.Handle.sctx, user, host, skipNameResolve) + record := mysqlPriv.matchIdentity(user, host, skipNameResolve) if record != nil { return record.User, record.Host, true } From 5ad1d086c24ae93b75c4ad9db1071f802e6202ae Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 28 Feb 2025 22:45:19 +0800 Subject: [PATCH 06/17] metrics: add metrics for active users count (#59533) ref pingcap/tidb#55563 --- br/pkg/restore/snap_client/pipeline_items.go | 2 +- pkg/domain/domain.go | 7 +- pkg/metrics/metrics.go | 1 + pkg/metrics/server.go | 9 + pkg/privilege/privileges/BUILD.bazel | 1 + pkg/privilege/privileges/cache.go | 36 +- pkg/session/bootstrap.go | 10 +- pkg/sessionctx/vardef/tidb_vars.go | 2020 ------------------ pkg/sessionctx/variable/tidb_vars.go | 5 + 9 files changed, 36 insertions(+), 2055 deletions(-) delete mode 100644 pkg/sessionctx/vardef/tidb_vars.go diff --git a/br/pkg/restore/snap_client/pipeline_items.go b/br/pkg/restore/snap_client/pipeline_items.go index 232847cbb31f4..0b6ded550e866 100644 --- a/br/pkg/restore/snap_client/pipeline_items.go +++ b/br/pkg/restore/snap_client/pipeline_items.go @@ -162,7 +162,7 @@ func (rc *SnapClient) replaceTables( return 0, errors.Trace(err) } - if err := notifyUpdateAllUsersPrivilege(renamedTables, rc.dom.NotifyUpdatePrivilege); err != nil { + if err := notifyUpdateAllUsersPrivilege(renamedTables, rc.dom.NotifyUpdateAllUsersPrivilege); err != nil { return 0, errors.Trace(err) } diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 51ee36002c49e..80b50a801279d 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1926,10 +1926,7 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { if err != nil { return err } - do.privHandle = privileges.NewHandle(sctx.GetRestrictedSQLExecutor()) - if err := do.privHandle.Update(nil); err != nil { - return errors.Trace(err) - } + do.privHandle = privileges.NewHandle(do.SysSessionPool(), sctx.GetSessionVars().GlobalVarsAccessor) var watchCh clientv3.WatchChan duration := 5 * time.Minute @@ -1981,7 +1978,7 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { func privReloadEvent(h *privileges.Handle, event *PrivilegeEvent) (err error) { switch { - case !vardef.AccelerateUserCreationUpdate.Load(): + case !variable.AccelerateUserCreationUpdate.Load(): err = h.UpdateAll() case event.All: err = h.UpdateAllActive() diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index c17859e256af8..2db78870d8da1 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -299,6 +299,7 @@ func RegisterMetrics() { prometheus.MustRegister(BindingCacheMemLimit) prometheus.MustRegister(BindingCacheNumBindings) prometheus.MustRegister(InternalSessions) + prometheus.MustRegister(ActiveUser) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/pkg/metrics/server.go b/pkg/metrics/server.go index 346f21ea54a16..be0795a97c83e 100644 --- a/pkg/metrics/server.go +++ b/pkg/metrics/server.go @@ -73,6 +73,7 @@ var ( RCCheckTSWriteConfilictCounter *prometheus.CounterVec MemoryLimit prometheus.Gauge InternalSessions prometheus.Gauge + ActiveUser prometheus.Gauge ) // InitServerMetrics initializes server metrics. @@ -402,6 +403,14 @@ func InitServerMetrics() { Name: "internal_sessions", Help: "The total count of internal sessions.", }) + + ActiveUser = prometheus.NewGauge( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "active_users", + Help: "The total count of active user.", + }) } // ExecuteErrorToLabel converts an execute error to label. diff --git a/pkg/privilege/privileges/BUILD.bazel b/pkg/privilege/privileges/BUILD.bazel index 72e7353d8f1a6..dd6aae40fc875 100644 --- a/pkg/privilege/privileges/BUILD.bazel +++ b/pkg/privilege/privileges/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/extension", "//pkg/infoschema", "//pkg/kv", + "//pkg/metrics", "//pkg/parser/auth", "//pkg/parser/mysql", "//pkg/parser/terror", diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 029ec12d23fb5..6b669a4d729a2 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -30,6 +30,7 @@ import ( "github.com/google/btree" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/metrics" "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" @@ -248,7 +249,7 @@ type defaultRoleRecord struct { // roleGraphEdgesTable is used to cache relationship between and role. type roleGraphEdgesTable struct { - roleList map[string]*auth.RoleIdentity + roleList map[auth.RoleIdentity]*auth.RoleIdentity } // Find method is used to find role from table @@ -256,7 +257,7 @@ func (g roleGraphEdgesTable) Find(user, host string) bool { if host == "" { host = "%" } - key := user + "@" + host + key := auth.RoleIdentity{user, host} if g.roleList == nil { return false } @@ -373,7 +374,7 @@ type MySQLPrivilege struct { globalPriv bTree[itemGlobalPriv] dynamicPriv bTree[itemDynamicPriv] - roleGraph map[string]roleGraphEdgesTable + roleGraph map[auth.RoleIdentity]roleGraphEdgesTable } func newMySQLPrivilege() *MySQLPrivilege { @@ -411,7 +412,7 @@ func (p *MySQLPrivilege) FindAllRole(activeRoles []*auth.RoleIdentity) []*auth.R if _, ok := visited[role.String()]; !ok { visited[role.String()] = true ret = append(ret, role) - key := role.Username + "@" + role.Hostname + key := auth.RoleIdentity{role.Username, role.Hostname} if edgeTable, ok := p.roleGraph[key]; ok { for _, v := range edgeTable.roleList { if _, ok := visited[v.String()]; !ok { @@ -430,7 +431,7 @@ func (p *MySQLPrivilege) FindRole(user string, host string, role *auth.RoleIdent rec := p.matchUser(user, host) r := p.matchUser(role.Username, role.Hostname) if rec != nil && r != nil { - key := rec.User + "@" + rec.Host + key := auth.RoleIdentity{rec.User, rec.Host} return p.roleGraph[key].Find(role.Username, role.Hostname) } return false @@ -986,7 +987,7 @@ func (p *MySQLPrivilege) decodeUserTableRow(userList map[string]struct{}) func(c var value UserRecord defaultAuthPlugin := "" if p.globalVars != nil { - val, err := p.globalVars.GetGlobalSysVar(vardef.DefaultAuthPlugin) + val, err := p.globalVars.GetGlobalSysVar(variable.DefaultAuthPlugin) if err == nil { defaultAuthPlugin = val } @@ -1063,8 +1064,6 @@ func (p *MySQLPrivilege) decodeUserTableRow(userList map[string]struct{}) func(c continue } value.PasswordLifeTime = row.GetInt64(i) - case f.ColumnAsName.L == "max_user_connections": - value.MaxUserConnections = row.GetInt64(i) case f.Column.GetType() == mysql.TypeEnum: if row.GetEnum(i).String() != "Y" { continue @@ -1248,11 +1247,11 @@ func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.Resul toUser = row.GetString(i) } } - fromKey := fromUser + "@" + fromHost - toKey := toUser + "@" + toHost + fromKey := auth.RoleIdentity{fromUser, fromHost} + toKey := auth.RoleIdentity{toUser, toHost} roleGraph, ok := p.roleGraph[toKey] if !ok { - roleGraph = roleGraphEdgesTable{roleList: make(map[string]*auth.RoleIdentity)} + roleGraph = roleGraphEdgesTable{roleList: make(map[auth.RoleIdentity]*auth.RoleIdentity)} p.roleGraph[toKey] = roleGraph } roleGraph.roleList[fromKey] = &auth.RoleIdentity{Username: fromUser, Hostname: fromHost} @@ -1831,15 +1830,13 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r slices.Sort(gs[sortFromIdx:]) // Show role grants. - graphKey := user + "@" + host + graphKey := auth.RoleIdentity{user, host} edgeTable, ok := p.roleGraph[graphKey] g = "" if ok { sortedRes := make([]string, 0, 10) for k := range edgeTable.roleList { - role := strings.Split(k, "@") - roleName, roleHost := role[0], role[1] - tmp := fmt.Sprintf("'%s'@'%s'", roleName, roleHost) + tmp := fmt.Sprintf("'%s'@'%s'", k.Username, k.Hostname) sortedRes = append(sortedRes, tmp) } slices.Sort(sortedRes) @@ -2066,7 +2063,7 @@ func (p *MySQLPrivilege) getDefaultRoles(user, host string) []*auth.RoleIdentity } func (p *MySQLPrivilege) getAllRoles(user, host string) []*auth.RoleIdentity { - key := user + "@" + host + key := auth.RoleIdentity{user, host} edgeTable, ok := p.roleGraph[key] ret := make([]*auth.RoleIdentity, 0, len(edgeTable.roleList)) if ok { @@ -2099,11 +2096,7 @@ func NewHandle(sctx util.SessionPool, globalVars variable.GlobalVarAccessor) *Ha } // ensureActiveUser ensure that the specific user data is loaded in-memory. -func (h *Handle) ensureActiveUser(ctx context.Context, user string) error { - if p := ctx.Value("mock"); p != nil { - visited := p.(*bool) - *visited = true - } +func (h *Handle) ensureActiveUser(user string) error { if h.fullData.Load() { // All users data are in-memory, nothing to do return nil @@ -2161,6 +2154,7 @@ func (h *Handle) UpdateAllActive() error { userList = append(userList, key.(string)) return true }) + metrics.ActiveUser.Set(float64(len(userList))) return h.updateUsers(userList) } diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 2e775190d9635..a3ad2c9ce872b 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -1269,7 +1269,7 @@ func DisableRunBootstrapSQLFileInTest() { } var ( - bootstrapVersion = []func(sessiontypes.Session, int64){ + bootstrapVersion = []func(sessiontypes.Session, int64) { upgradeToVer2, upgradeToVer3, upgradeToVer4, @@ -1427,17 +1427,12 @@ var ( upgradeToVer216, upgradeToVer217, upgradeToVer218, -<<<<<<< HEAD upgradeToVer219, upgradeToVer220, upgradeToVer221, upgradeToVer222, upgradeToVer223, -======= - upgradeToVer239, - upgradeToVer240, - upgradeToVer241, ->>>>>>> 8e1a47dddb (session: add index on 'user' field for mysql tables (#57470)) + upgradeToVer224, } ) @@ -3282,7 +3277,6 @@ func upgradeToVer219(s sessiontypes.Session, ver int64) { doReentrantDDL(s, addAnalyzeJobsSchemaTablePartitionStateIndex, dbterror.ErrDupKeyName) } -<<<<<<< HEAD func upgradeToVer220(s sessiontypes.Session, ver int64) { if ver >= version220 { return diff --git a/pkg/sessionctx/vardef/tidb_vars.go b/pkg/sessionctx/vardef/tidb_vars.go deleted file mode 100644 index 6efe98cb3ffd7..0000000000000 --- a/pkg/sessionctx/vardef/tidb_vars.go +++ /dev/null @@ -1,2020 +0,0 @@ -// Copyright 2025 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package vardef - -import ( - "fmt" - "math" - "strconv" - "strings" - goatomic "sync/atomic" - "time" - - "github.com/pingcap/tidb/pkg/config" - "github.com/pingcap/tidb/pkg/executor/join/joinversion" - "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/util/memory" - "github.com/pingcap/tidb/pkg/util/paging" - "github.com/pingcap/tidb/pkg/util/size" - "github.com/pingcap/tipb/go-tipb" - "go.uber.org/atomic" -) - -/* - Steps to add a new TiDB specific system variable: - - 1. Add a new variable name with comment in this file. - 2. Add the default value of the new variable in this file. - 3. Add SysVar instance in 'defaultSysVars' slice. -*/ - -// TiDB system variable names that only in session scope. -const ( - TiDBDDLSlowOprThreshold = "ddl_slow_threshold" - - // TiDBSnapshot is used for reading history data, the default value is empty string. - // The value can be a datetime string like '2017-11-11 20:20:20' or a tso string. When this variable is set, the session reads history data of that time. - TiDBSnapshot = "tidb_snapshot" - - // TiDBOptAggPushDown is used to enable/disable the optimizer rule of aggregation push down. - TiDBOptAggPushDown = "tidb_opt_agg_push_down" - - // TiDBOptDeriveTopN is used to enable/disable the optimizer rule of deriving topN. - TiDBOptDeriveTopN = "tidb_opt_derive_topn" - - // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode - TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" - - TiDBOptMPPOuterJoinFixedBuildSide = "tidb_opt_mpp_outer_join_fixed_build_side" - - // TiDBOptDistinctAggPushDown is used to decide whether agg with distinct should be pushed to tikv/tiflash. - TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" - - // TiDBOptSkewDistinctAgg is used to indicate the distinct agg has data skew - TiDBOptSkewDistinctAgg = "tidb_opt_skew_distinct_agg" - - // TiDBOpt3StageDistinctAgg is used to indicate whether to plan and execute the distinct agg in 3 stages - TiDBOpt3StageDistinctAgg = "tidb_opt_three_stage_distinct_agg" - - // TiDBOptEnable3StageMultiDistinctAgg is used to indicate whether to plan and execute the multi distinct agg in 3 stages - TiDBOptEnable3StageMultiDistinctAgg = "tidb_opt_enable_three_stage_multi_distinct_agg" - - TiDBOptExplainNoEvaledSubQuery = "tidb_opt_enable_non_eval_scalar_subquery" - - // TiDBBCJThresholdSize is used to limit the size of small table for mpp broadcast join. - // Its unit is bytes, if the size of small table is larger than it, we will not use bcj. - TiDBBCJThresholdSize = "tidb_broadcast_join_threshold_size" - - // TiDBBCJThresholdCount is used to limit the count of small table for mpp broadcast join. - // If we can't estimate the size of one side of join child, we will check if its row number exceeds this limitation. - TiDBBCJThresholdCount = "tidb_broadcast_join_threshold_count" - - // TiDBPreferBCJByExchangeDataSize indicates the method used to choose mpp broadcast join - TiDBPreferBCJByExchangeDataSize = "tidb_prefer_broadcast_join_by_exchange_data_size" - - // TiDBOptWriteRowID is used to enable/disable the operations of insert、replace and update to _tidb_rowid. - TiDBOptWriteRowID = "tidb_opt_write_row_id" - - // TiDBAutoAnalyzeRatio will run if (table modify count)/(table row count) is greater than this value. - TiDBAutoAnalyzeRatio = "tidb_auto_analyze_ratio" - - // TiDBAutoAnalyzeStartTime will run if current time is within start time and end time. - TiDBAutoAnalyzeStartTime = "tidb_auto_analyze_start_time" - TiDBAutoAnalyzeEndTime = "tidb_auto_analyze_end_time" - - // TiDBChecksumTableConcurrency is used to speed up the ADMIN CHECKSUM TABLE - // statement, when a table has multiple indices, those indices can be - // scanned concurrently, with the cost of higher system performance impact. - TiDBChecksumTableConcurrency = "tidb_checksum_table_concurrency" - - // TiDBCurrentTS is used to get the current transaction timestamp. - // It is read-only. - TiDBCurrentTS = "tidb_current_ts" - - // TiDBLastTxnInfo is used to get the last transaction info within the current session. - TiDBLastTxnInfo = "tidb_last_txn_info" - - // TiDBLastQueryInfo is used to get the last query info within the current session. - TiDBLastQueryInfo = "tidb_last_query_info" - - // TiDBLastDDLInfo is used to get the last ddl info within the current session. - TiDBLastDDLInfo = "tidb_last_ddl_info" - - // TiDBLastPlanReplayerToken is used to get the last plan replayer token within the current session - TiDBLastPlanReplayerToken = "tidb_last_plan_replayer_token" - - // TiDBConfig is a read-only variable that shows the config of the current server. - TiDBConfig = "tidb_config" - - // TiDBBatchInsert is used to enable/disable auto-split insert data. If set this option on, insert executor will automatically - // insert data into multiple batches and use a single txn for each batch. This will be helpful when inserting large data. - TiDBBatchInsert = "tidb_batch_insert" - - // TiDBBatchDelete is used to enable/disable auto-split delete data. If set this option on, delete executor will automatically - // split data into multiple batches and use a single txn for each batch. This will be helpful when deleting large data. - TiDBBatchDelete = "tidb_batch_delete" - - // TiDBBatchCommit is used to enable/disable auto-split the transaction. - // If set this option on, the transaction will be committed when it reaches stmt-count-limit and starts a new transaction. - TiDBBatchCommit = "tidb_batch_commit" - - // TiDBDMLBatchSize is used to split the insert/delete data into small batches. - // It only takes effort when tidb_batch_insert/tidb_batch_delete is on. - // Its default value is 20000. When the row size is large, 20k rows could be larger than 100MB. - // User could change it to a smaller one to avoid breaking the transaction size limitation. - TiDBDMLBatchSize = "tidb_dml_batch_size" - - // The following session variables controls the memory quota during query execution. - - // TiDBMemQuotaQuery controls the memory quota of a query. - TiDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. - // TiDBMemQuotaApplyCache controls the memory quota of a query. - TiDBMemQuotaApplyCache = "tidb_mem_quota_apply_cache" - - // TiDBGeneralLog is used to log every query in the server in info level. - TiDBGeneralLog = "tidb_general_log" - - // TiDBLogFileMaxDays is used to log every query in the server in info level. - TiDBLogFileMaxDays = "tidb_log_file_max_days" - - // TiDBPProfSQLCPU is used to add label sql label to pprof result. - TiDBPProfSQLCPU = "tidb_pprof_sql_cpu" - - // TiDBRetryLimit is the maximum number of retries when committing a transaction. - TiDBRetryLimit = "tidb_retry_limit" - - // TiDBDisableTxnAutoRetry disables transaction auto retry. - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBDisableTxnAutoRetry = "tidb_disable_txn_auto_retry" - - // TiDBEnableChunkRPC enables TiDB to use Chunk format for coprocessor requests. - TiDBEnableChunkRPC = "tidb_enable_chunk_rpc" - - // TiDBOptimizerSelectivityLevel is used to control the selectivity estimation level. - TiDBOptimizerSelectivityLevel = "tidb_optimizer_selectivity_level" - - // TiDBOptimizerEnableNewOnlyFullGroupByCheck is used to open the newly only_full_group_by check by maintaining functional dependency. - TiDBOptimizerEnableNewOnlyFullGroupByCheck = "tidb_enable_new_only_full_group_by_check" - - TiDBOptimizerEnableOuterJoinReorder = "tidb_enable_outer_join_reorder" - - // TiDBOptimizerEnableNAAJ is used to open the newly null-aware anti join - TiDBOptimizerEnableNAAJ = "tidb_enable_null_aware_anti_join" - - // TiDBTxnMode is used to control the transaction behavior. - TiDBTxnMode = "tidb_txn_mode" - - // TiDBRowFormatVersion is used to control tidb row format version current. - TiDBRowFormatVersion = "tidb_row_format_version" - - // TiDBEnableRowLevelChecksum is used to control whether to append checksum to row values. - TiDBEnableRowLevelChecksum = "tidb_enable_row_level_checksum" - - // TiDBEnableTablePartition is used to control table partition feature. - // The valid value include auto/on/off: - // on or auto: enable table partition if the partition type is implemented. - // off: always disable table partition. - TiDBEnableTablePartition = "tidb_enable_table_partition" - - // TiDBEnableListTablePartition is used to control list table partition feature. - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBEnableListTablePartition = "tidb_enable_list_partition" - - // TiDBSkipIsolationLevelCheck is used to control whether to return error when set unsupported transaction - // isolation level. - TiDBSkipIsolationLevelCheck = "tidb_skip_isolation_level_check" - - // TiDBLowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds - TiDBLowResolutionTSO = "tidb_low_resolution_tso" - - // TiDBReplicaRead is used for reading data from replicas, followers for example. - TiDBReplicaRead = "tidb_replica_read" - - // TiDBAdaptiveClosestReadThreshold is for reading data from closest replicas(with same 'zone' label). - // TiKV client should send read request to the closest replica(leader/follower) if the estimated response - // size exceeds this threshold; otherwise, this request should be sent to leader. - // This variable only take effect when `tidb_replica_read` is 'closest-adaptive'. - TiDBAdaptiveClosestReadThreshold = "tidb_adaptive_closest_read_threshold" - - // TiDBAllowRemoveAutoInc indicates whether a user can drop the auto_increment column attribute or not. - TiDBAllowRemoveAutoInc = "tidb_allow_remove_auto_inc" - - // TiDBMultiStatementMode enables multi statement at the risk of SQL injection - // provides backwards compatibility - TiDBMultiStatementMode = "tidb_multi_statement_mode" - - // TiDBEvolvePlanTaskMaxTime controls the max time of a single evolution task. - TiDBEvolvePlanTaskMaxTime = "tidb_evolve_plan_task_max_time" - - // TiDBEvolvePlanTaskStartTime is the start time of evolution task. - TiDBEvolvePlanTaskStartTime = "tidb_evolve_plan_task_start_time" - // TiDBEvolvePlanTaskEndTime is the end time of evolution task. - TiDBEvolvePlanTaskEndTime = "tidb_evolve_plan_task_end_time" - - // TiDBSlowLogThreshold is used to set the slow log threshold in the server. - TiDBSlowLogThreshold = "tidb_slow_log_threshold" - - // TiDBSlowTxnLogThreshold is used to set the slow transaction log threshold in the server. - TiDBSlowTxnLogThreshold = "tidb_slow_txn_log_threshold" - - // TiDBRecordPlanInSlowLog is used to log the plan of the slow query. - TiDBRecordPlanInSlowLog = "tidb_record_plan_in_slow_log" - - // TiDBEnableSlowLog enables TiDB to log slow queries. - TiDBEnableSlowLog = "tidb_enable_slow_log" - - // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. - TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" - - // TiDBFoundInPlanCache indicates whether the last statement was found in plan cache - TiDBFoundInPlanCache = "last_plan_from_cache" - - // TiDBFoundInBinding indicates whether the last statement was matched with the hints in the binding. - TiDBFoundInBinding = "last_plan_from_binding" - - // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. - TiDBAllowAutoRandExplicitInsert = "allow_auto_random_explicit_insert" - - // TiDBTxnReadTS indicates the next transaction should be staleness transaction and provide the startTS - TiDBTxnReadTS = "tx_read_ts" - - // TiDBReadStaleness indicates the staleness duration for following statement - TiDBReadStaleness = "tidb_read_staleness" - - // TiDBEnablePaging indicates whether paging is enabled in coprocessor requests. - TiDBEnablePaging = "tidb_enable_paging" - - // TiDBReadConsistency indicates whether the autocommit read statement goes through TiKV RC. - TiDBReadConsistency = "tidb_read_consistency" - - // TiDBSysdateIsNow is the name of the `tidb_sysdate_is_now` system variable - TiDBSysdateIsNow = "tidb_sysdate_is_now" - - // RequireSecureTransport indicates the secure mode for data transport - RequireSecureTransport = "require_secure_transport" - - // TiFlashFastScan indicates whether use fast scan in tiflash. - TiFlashFastScan = "tiflash_fastscan" - - // TiDBEnableUnsafeSubstitute indicates whether to enable generate column takes unsafe substitute. - TiDBEnableUnsafeSubstitute = "tidb_enable_unsafe_substitute" - - // TiDBEnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. - TiDBEnableTiFlashReadForWriteStmt = "tidb_enable_tiflash_read_for_write_stmt" - - // TiDBUseAlloc indicates whether the last statement used chunk alloc - TiDBUseAlloc = "last_sql_use_alloc" - - // TiDBExplicitRequestSourceType indicates the source of the request, it's a complement of RequestSourceType. - // The value maybe "lightning", "br", "dumpling" etc. - TiDBExplicitRequestSourceType = "tidb_request_source_type" -) - -// TiDB system variable names that both in session and global scope. -const ( - // TiDBBuildStatsConcurrency specifies the number of concurrent workers used for analyzing tables or partitions. - // When multiple tables or partitions are specified in the analyze statement, TiDB will process them concurrently. - // Additionally, this setting controls the concurrency for building NDV (Number of Distinct Values) for special indexes, - // such as generated columns composed indexes. - TiDBBuildStatsConcurrency = "tidb_build_stats_concurrency" - - // TiDBBuildSamplingStatsConcurrency is used to control the concurrency of building stats using sampling. - // 1. The number of concurrent workers to merge FMSketches and Sample Data from different regions. - // 2. The number of concurrent workers to build TopN and Histogram concurrently. - TiDBBuildSamplingStatsConcurrency = "tidb_build_sampling_stats_concurrency" - - // TiDBDistSQLScanConcurrency is used to set the concurrency of a distsql scan task. - // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. - // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. - // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. - TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" - - // TiDBAnalyzeDistSQLScanConcurrency is the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). - // For auto analyze, the value is controlled by tidb_sysproc_scan_concurrency variable. - TiDBAnalyzeDistSQLScanConcurrency = "tidb_analyze_distsql_scan_concurrency" - - // TiDBOptInSubqToJoinAndAgg is used to enable/disable the optimizer rule of rewriting IN subquery. - TiDBOptInSubqToJoinAndAgg = "tidb_opt_insubq_to_join_and_agg" - - // TiDBOptPreferRangeScan is used to enable/disable the optimizer to always prefer range scan over table scan, ignoring their costs. - TiDBOptPreferRangeScan = "tidb_opt_prefer_range_scan" - - // TiDBOptEnableCorrelationAdjustment is used to indicates if enable correlation adjustment. - TiDBOptEnableCorrelationAdjustment = "tidb_opt_enable_correlation_adjustment" - - // TiDBOptLimitPushDownThreshold determines if push Limit or TopN down to TiKV forcibly. - TiDBOptLimitPushDownThreshold = "tidb_opt_limit_push_down_threshold" - - // TiDBOptCorrelationThreshold is a guard to enable row count estimation using column order correlation. - TiDBOptCorrelationThreshold = "tidb_opt_correlation_threshold" - - // TiDBOptCorrelationExpFactor is an exponential factor to control heuristic approach when tidb_opt_correlation_threshold is not satisfied. - TiDBOptCorrelationExpFactor = "tidb_opt_correlation_exp_factor" - - // TiDBOptCPUFactor is the CPU cost of processing one expression for one row. - TiDBOptCPUFactor = "tidb_opt_cpu_factor" - // TiDBOptCopCPUFactor is the CPU cost of processing one expression for one row in coprocessor. - TiDBOptCopCPUFactor = "tidb_opt_copcpu_factor" - // TiDBOptTiFlashConcurrencyFactor is concurrency number of tiflash computation. - TiDBOptTiFlashConcurrencyFactor = "tidb_opt_tiflash_concurrency_factor" - // TiDBOptNetworkFactor is the network cost of transferring 1 byte data. - TiDBOptNetworkFactor = "tidb_opt_network_factor" - // TiDBOptScanFactor is the IO cost of scanning 1 byte data on TiKV. - TiDBOptScanFactor = "tidb_opt_scan_factor" - // TiDBOptDescScanFactor is the IO cost of scanning 1 byte data on TiKV in desc order. - TiDBOptDescScanFactor = "tidb_opt_desc_factor" - // TiDBOptSeekFactor is the IO cost of seeking the start value in a range on TiKV or TiFlash. - TiDBOptSeekFactor = "tidb_opt_seek_factor" - // TiDBOptMemoryFactor is the memory cost of storing one tuple. - TiDBOptMemoryFactor = "tidb_opt_memory_factor" - // TiDBOptDiskFactor is the IO cost of reading/writing one byte to temporary disk. - TiDBOptDiskFactor = "tidb_opt_disk_factor" - // TiDBOptConcurrencyFactor is the CPU cost of additional one goroutine. - TiDBOptConcurrencyFactor = "tidb_opt_concurrency_factor" - // TiDBOptForceInlineCTE is used to enable/disable inline CTE - TiDBOptForceInlineCTE = "tidb_opt_force_inline_cte" - - // TiDBIndexJoinBatchSize is used to set the batch size of an index lookup join. - // The index lookup join fetches batches of data from outer executor and constructs ranges for inner executor. - // This value controls how much of data in a batch to do the index join. - // Large value may reduce the latency but consumes more system resource. - TiDBIndexJoinBatchSize = "tidb_index_join_batch_size" - - // TiDBIndexLookupSize is used for index lookup executor. - // The index lookup executor first scan a batch of handles from a index, then use those handles to lookup the table - // rows, this value controls how much of handles in a batch to do a lookup task. - // Small value sends more RPCs to TiKV, consume more system resource. - // Large value may do more work than needed if the query has a limit. - TiDBIndexLookupSize = "tidb_index_lookup_size" - - // TiDBIndexLookupConcurrency is used for index lookup executor. - // A lookup task may have 'tidb_index_lookup_size' of handles at maximum, the handles may be distributed - // in many TiKV nodes, we execute multiple concurrent index lookup tasks concurrently to reduce the time - // waiting for a task to finish. - // Set this value higher may reduce the latency but consumes more system resource. - // tidb_index_lookup_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBIndexLookupConcurrency = "tidb_index_lookup_concurrency" - - // TiDBIndexLookupJoinConcurrency is used for index lookup join executor. - // IndexLookUpJoin starts "tidb_index_lookup_join_concurrency" inner workers - // to fetch inner rows and join the matched (outer, inner) row pairs. - // tidb_index_lookup_join_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBIndexLookupJoinConcurrency = "tidb_index_lookup_join_concurrency" - - // TiDBIndexSerialScanConcurrency is used for controlling the concurrency of index scan operation - // when we need to keep the data output order the same as the order of index data. - TiDBIndexSerialScanConcurrency = "tidb_index_serial_scan_concurrency" - - // TiDBMaxChunkSize is used to control the max chunk size during query execution. - TiDBMaxChunkSize = "tidb_max_chunk_size" - - // TiDBAllowBatchCop means if we should send batch coprocessor to TiFlash. It can be set to 0, 1 and 2. - // 0 means never use batch cop, 1 means use batch cop in case of aggregation and join, 2, means to force sending batch cop for any query. - // The default value is 0 - TiDBAllowBatchCop = "tidb_allow_batch_cop" - - // TiDBShardRowIDBits means all the tables created in the current session will be sharded. - // The default value is 0 - TiDBShardRowIDBits = "tidb_shard_row_id_bits" - - // TiDBPreSplitRegions means all the tables created in the current session will be pre-splited. - // The default value is 0 - TiDBPreSplitRegions = "tidb_pre_split_regions" - - // TiDBAllowMPPExecution means if we should use mpp way to execute query or not. - // Default value is `true`, means to be determined by the optimizer. - // Value set to `false` means never use mpp. - TiDBAllowMPPExecution = "tidb_allow_mpp" - - // TiDBAllowTiFlashCop means we only use MPP mode to query data. - // Default value is `true`, means to be determined by the optimizer. - // Value set to `false` means we may fall back to TiFlash cop plan if possible. - TiDBAllowTiFlashCop = "tidb_allow_tiflash_cop" - - // TiDBHashExchangeWithNewCollation means if hash exchange is supported when new collation is on. - // Default value is `true`, means support hash exchange when new collation is on. - // Value set to `false` means not support hash exchange when new collation is on. - TiDBHashExchangeWithNewCollation = "tidb_hash_exchange_with_new_collation" - - // TiDBEnforceMPPExecution means if we should enforce mpp way to execute query or not. - // Default value is `false`, means to be determined by variable `tidb_allow_mpp`. - // Value set to `true` means enforce use mpp. - // Note if you want to set `tidb_enforce_mpp` to `true`, you must set `tidb_allow_mpp` to `true` first. - TiDBEnforceMPPExecution = "tidb_enforce_mpp" - - // TiDBMaxTiFlashThreads is the maximum number of threads to execute the request which is pushed down to tiflash. - // Default value is -1, means it will not be pushed down to tiflash. - // If the value is bigger than -1, it will be pushed down to tiflash and used to create db context in tiflash. - TiDBMaxTiFlashThreads = "tidb_max_tiflash_threads" - - // TiDBMaxBytesBeforeTiFlashExternalJoin is the maximum bytes used by a TiFlash join before spill to disk - TiDBMaxBytesBeforeTiFlashExternalJoin = "tidb_max_bytes_before_tiflash_external_join" - - // TiDBMaxBytesBeforeTiFlashExternalGroupBy is the maximum bytes used by a TiFlash hash aggregation before spill to disk - TiDBMaxBytesBeforeTiFlashExternalGroupBy = "tidb_max_bytes_before_tiflash_external_group_by" - - // TiDBMaxBytesBeforeTiFlashExternalSort is the maximum bytes used by a TiFlash sort/TopN before spill to disk - TiDBMaxBytesBeforeTiFlashExternalSort = "tidb_max_bytes_before_tiflash_external_sort" - - // TiFlashMemQuotaQueryPerNode is the maximum bytes used by a TiFlash Query on each TiFlash node - TiFlashMemQuotaQueryPerNode = "tiflash_mem_quota_query_per_node" - - // TiFlashQuerySpillRatio is the threshold that TiFlash will trigger auto spill when the memory usage is above this percentage - TiFlashQuerySpillRatio = "tiflash_query_spill_ratio" - - // TiDBMPPStoreFailTTL is the unavailable time when a store is detected failed. During that time, tidb will not send any task to - // TiFlash even though the failed TiFlash node has been recovered. - TiDBMPPStoreFailTTL = "tidb_mpp_store_fail_ttl" - - // TiDBInitChunkSize is used to control the init chunk size during query execution. - TiDBInitChunkSize = "tidb_init_chunk_size" - - // TiDBMinPagingSize is used to control the min paging size in the coprocessor paging protocol. - TiDBMinPagingSize = "tidb_min_paging_size" - - // TiDBMaxPagingSize is used to control the max paging size in the coprocessor paging protocol. - TiDBMaxPagingSize = "tidb_max_paging_size" - - // TiDBEnableCascadesPlanner is used to control whether to enable the cascades planner. - TiDBEnableCascadesPlanner = "tidb_enable_cascades_planner" - - // TiDBSkipUTF8Check skips the UTF8 validate process, validate UTF8 has performance cost, if we can make sure - // the input string values are valid, we can skip the check. - TiDBSkipUTF8Check = "tidb_skip_utf8_check" - - // TiDBSkipASCIICheck skips the ASCII validate process - // old tidb may already have fields with invalid ASCII bytes - // disable ASCII validate can guarantee a safe replication - TiDBSkipASCIICheck = "tidb_skip_ascii_check" - - // TiDBHashJoinConcurrency is used for hash join executor. - // The hash join outer executor starts multiple concurrent join workers to probe the hash table. - // tidb_hash_join_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBHashJoinConcurrency = "tidb_hash_join_concurrency" - - // TiDBProjectionConcurrency is used for projection operator. - // This variable controls the worker number of projection operator. - // tidb_projection_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBProjectionConcurrency = "tidb_projection_concurrency" - - // TiDBHashAggPartialConcurrency is used for hash agg executor. - // The hash agg executor starts multiple concurrent partial workers to do partial aggregate works. - // tidb_hashagg_partial_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBHashAggPartialConcurrency = "tidb_hashagg_partial_concurrency" - - // TiDBHashAggFinalConcurrency is used for hash agg executor. - // The hash agg executor starts multiple concurrent final workers to do final aggregate works. - // tidb_hashagg_final_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBHashAggFinalConcurrency = "tidb_hashagg_final_concurrency" - - // TiDBWindowConcurrency is used for window parallel executor. - // tidb_window_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBWindowConcurrency = "tidb_window_concurrency" - - // TiDBMergeJoinConcurrency is used for merge join parallel executor - TiDBMergeJoinConcurrency = "tidb_merge_join_concurrency" - - // TiDBStreamAggConcurrency is used for stream aggregation parallel executor. - // tidb_stream_agg_concurrency is deprecated, use tidb_executor_concurrency instead. - TiDBStreamAggConcurrency = "tidb_streamagg_concurrency" - - // TiDBIndexMergeIntersectionConcurrency is used for parallel worker of index merge intersection. - TiDBIndexMergeIntersectionConcurrency = "tidb_index_merge_intersection_concurrency" - - // TiDBEnableParallelApply is used for parallel apply. - TiDBEnableParallelApply = "tidb_enable_parallel_apply" - - // TiDBBackoffLockFast is used for tikv backoff base time in milliseconds. - TiDBBackoffLockFast = "tidb_backoff_lock_fast" - - // TiDBBackOffWeight is used to control the max back off time in TiDB. - // The default maximum back off time is a small value. - // BackOffWeight could multiply it to let the user adjust the maximum time for retrying. - // Only positive integers can be accepted, which means that the maximum back off time can only grow. - TiDBBackOffWeight = "tidb_backoff_weight" - - // TiDBDDLReorgWorkerCount defines the count of ddl reorg workers. - TiDBDDLReorgWorkerCount = "tidb_ddl_reorg_worker_cnt" - - // TiDBDDLFlashbackConcurrency defines the count of ddl flashback workers. - TiDBDDLFlashbackConcurrency = "tidb_ddl_flashback_concurrency" - - // TiDBDDLReorgBatchSize defines the transaction batch size of ddl reorg workers. - TiDBDDLReorgBatchSize = "tidb_ddl_reorg_batch_size" - - // TiDBDDLErrorCountLimit defines the count of ddl error limit. - TiDBDDLErrorCountLimit = "tidb_ddl_error_count_limit" - - // TiDBDDLReorgPriority defines the operations' priority of adding indices. - // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH - TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" - - // TiDBDDLReorgMaxWriteSpeed defines the max write limitation for the lightning local backend - TiDBDDLReorgMaxWriteSpeed = "tidb_ddl_reorg_max_write_speed" - - // TiDBEnableAutoIncrementInGenerated disables the mysql compatibility check on using auto-incremented columns in - // expression indexes and generated columns described here https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details. - TiDBEnableAutoIncrementInGenerated = "tidb_enable_auto_increment_in_generated" - - // TiDBPlacementMode is used to control the mode for placement - TiDBPlacementMode = "tidb_placement_mode" - - // TiDBMaxDeltaSchemaCount defines the max length of deltaSchemaInfos. - // deltaSchemaInfos is a queue that maintains the history of schema changes. - TiDBMaxDeltaSchemaCount = "tidb_max_delta_schema_count" - - // TiDBScatterRegion will scatter the regions for DDLs when it is "table" or "global", "" indicates not trigger scatter. - TiDBScatterRegion = "tidb_scatter_region" - - // TiDBWaitSplitRegionFinish defines the split region behaviour is sync or async. - TiDBWaitSplitRegionFinish = "tidb_wait_split_region_finish" - - // TiDBWaitSplitRegionTimeout uses to set the split and scatter region back off time. - TiDBWaitSplitRegionTimeout = "tidb_wait_split_region_timeout" - - // TiDBForcePriority defines the operations' priority of all statements. - // It can be "NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED" - TiDBForcePriority = "tidb_force_priority" - - // TiDBConstraintCheckInPlace indicates to check the constraint when the SQL executing. - // It could hurt the performance of bulking insert when it is ON. - TiDBConstraintCheckInPlace = "tidb_constraint_check_in_place" - - // TiDBEnableWindowFunction is used to control whether to enable the window function. - TiDBEnableWindowFunction = "tidb_enable_window_function" - - // TiDBEnablePipelinedWindowFunction is used to control whether to use pipelined window function, it only works when tidb_enable_window_function = true. - TiDBEnablePipelinedWindowFunction = "tidb_enable_pipelined_window_function" - - // TiDBEnableStrictDoubleTypeCheck is used to control table field double type syntax check. - TiDBEnableStrictDoubleTypeCheck = "tidb_enable_strict_double_type_check" - - // TiDBOptProjectionPushDown is used to control whether to pushdown projection to coprocessor. - TiDBOptProjectionPushDown = "tidb_opt_projection_push_down" - - // TiDBEnableVectorizedExpression is used to control whether to enable the vectorized expression evaluation. - TiDBEnableVectorizedExpression = "tidb_enable_vectorized_expression" - - // TiDBOptJoinReorderThreshold defines the threshold less than which - // we'll choose a rather time-consuming algorithm to calculate the join order. - TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" - - // TiDBSlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. - TiDBSlowQueryFile = "tidb_slow_query_file" - - // TiDBEnableFastAnalyze indicates to use fast analyze. - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBEnableFastAnalyze = "tidb_enable_fast_analyze" - - // TiDBExpensiveQueryTimeThreshold indicates the time threshold of expensive query. - TiDBExpensiveQueryTimeThreshold = "tidb_expensive_query_time_threshold" - - // TiDBExpensiveTxnTimeThreshold indicates the time threshold of expensive transaction. - TiDBExpensiveTxnTimeThreshold = "tidb_expensive_txn_time_threshold" - - // TiDBEnableIndexMerge indicates to generate IndexMergePath. - TiDBEnableIndexMerge = "tidb_enable_index_merge" - - // TiDBEnableNoopFuncs set true will enable using fake funcs(like get_lock release_lock) - TiDBEnableNoopFuncs = "tidb_enable_noop_functions" - - // TiDBEnableStmtSummary indicates whether the statement summary is enabled. - TiDBEnableStmtSummary = "tidb_enable_stmt_summary" - - // TiDBStmtSummaryInternalQuery indicates whether the statement summary contain internal query. - TiDBStmtSummaryInternalQuery = "tidb_stmt_summary_internal_query" - - // TiDBStmtSummaryRefreshInterval indicates the refresh interval in seconds for each statement summary. - TiDBStmtSummaryRefreshInterval = "tidb_stmt_summary_refresh_interval" - - // TiDBStmtSummaryHistorySize indicates the history size of each statement summary. - TiDBStmtSummaryHistorySize = "tidb_stmt_summary_history_size" - - // TiDBStmtSummaryMaxStmtCount indicates the max number of statements kept in memory. - TiDBStmtSummaryMaxStmtCount = "tidb_stmt_summary_max_stmt_count" - - // TiDBStmtSummaryMaxSQLLength indicates the max length of displayed normalized sql and sample sql. - TiDBStmtSummaryMaxSQLLength = "tidb_stmt_summary_max_sql_length" - - // TiDBIgnoreInlistPlanDigest enables TiDB to generate the same plan digest with SQL using different in-list arguments. - TiDBIgnoreInlistPlanDigest = "tidb_ignore_inlist_plan_digest" - - // TiDBCapturePlanBaseline indicates whether the capture of plan baselines is enabled. - TiDBCapturePlanBaseline = "tidb_capture_plan_baselines" - - // TiDBUsePlanBaselines indicates whether the use of plan baselines is enabled. - TiDBUsePlanBaselines = "tidb_use_plan_baselines" - - // TiDBEvolvePlanBaselines indicates whether the evolution of plan baselines is enabled. - TiDBEvolvePlanBaselines = "tidb_evolve_plan_baselines" - - // TiDBOptEnableFuzzyBinding indicates whether to enable the universal binding. - TiDBOptEnableFuzzyBinding = "tidb_opt_enable_fuzzy_binding" - - // TiDBEnableExtendedStats indicates whether the extended statistics feature is enabled. - TiDBEnableExtendedStats = "tidb_enable_extended_stats" - - // TiDBIsolationReadEngines indicates the tidb only read from the stores whose engine type is involved in IsolationReadEngines. - // Now, only support TiKV and TiFlash. - TiDBIsolationReadEngines = "tidb_isolation_read_engines" - - // TiDBStoreLimit indicates the limit of sending request to a store, 0 means without limit. - TiDBStoreLimit = "tidb_store_limit" - - // TiDBMetricSchemaStep indicates the step when query metric schema. - TiDBMetricSchemaStep = "tidb_metric_query_step" - - // TiDBCDCWriteSource indicates the following data is written by TiCDC if it is not 0. - TiDBCDCWriteSource = "tidb_cdc_write_source" - - // TiDBMetricSchemaRangeDuration indicates the range duration when query metric schema. - TiDBMetricSchemaRangeDuration = "tidb_metric_query_range_duration" - - // TiDBEnableCollectExecutionInfo indicates that whether execution info is collected. - TiDBEnableCollectExecutionInfo = "tidb_enable_collect_execution_info" - - // TiDBExecutorConcurrency is used for controlling the concurrency of all types of executors. - TiDBExecutorConcurrency = "tidb_executor_concurrency" - - // TiDBEnableClusteredIndex indicates if clustered index feature is enabled. - TiDBEnableClusteredIndex = "tidb_enable_clustered_index" - - // TiDBEnableGlobalIndex means if we could create an global index on a partition table or not. - // Deprecated, will always be ON - TiDBEnableGlobalIndex = "tidb_enable_global_index" - - // TiDBPartitionPruneMode indicates the partition prune mode used. - TiDBPartitionPruneMode = "tidb_partition_prune_mode" - - // TiDBRedactLog indicates that whether redact log. - TiDBRedactLog = "tidb_redact_log" - - // TiDBRestrictedReadOnly is meant for the cloud admin to toggle the cluster read only - TiDBRestrictedReadOnly = "tidb_restricted_read_only" - - // TiDBSuperReadOnly is tidb's variant of mysql's super_read_only, which has some differences from mysql's super_read_only. - TiDBSuperReadOnly = "tidb_super_read_only" - - // TiDBShardAllocateStep indicates the max size of continuous rowid shard in one transaction. - TiDBShardAllocateStep = "tidb_shard_allocate_step" - // TiDBEnableTelemetry indicates that whether usage data report to PingCAP is enabled. - // Deprecated: it is 'off' always since Telemetry has been removed from TiDB. - TiDBEnableTelemetry = "tidb_enable_telemetry" - - // TiDBMemoryUsageAlarmRatio indicates the alarm threshold when memory usage of the tidb-server exceeds. - TiDBMemoryUsageAlarmRatio = "tidb_memory_usage_alarm_ratio" - - // TiDBMemoryUsageAlarmKeepRecordNum indicates the number of saved alarm files. - TiDBMemoryUsageAlarmKeepRecordNum = "tidb_memory_usage_alarm_keep_record_num" - - // TiDBEnableRateLimitAction indicates whether enabled ratelimit action - TiDBEnableRateLimitAction = "tidb_enable_rate_limit_action" - - // TiDBEnableAsyncCommit indicates whether to enable the async commit feature. - TiDBEnableAsyncCommit = "tidb_enable_async_commit" - - // TiDBEnable1PC indicates whether to enable the one-phase commit feature. - TiDBEnable1PC = "tidb_enable_1pc" - - // TiDBGuaranteeLinearizability indicates whether to guarantee linearizability. - TiDBGuaranteeLinearizability = "tidb_guarantee_linearizability" - - // TiDBAnalyzeVersion indicates how tidb collects the analyzed statistics and how use to it. - TiDBAnalyzeVersion = "tidb_analyze_version" - - // TiDBAutoAnalyzePartitionBatchSize indicates the batch size for partition tables for auto analyze in dynamic mode - // Deprecated: This variable is deprecated, please do not use this variable. - TiDBAutoAnalyzePartitionBatchSize = "tidb_auto_analyze_partition_batch_size" - - // TiDBEnableIndexMergeJoin indicates whether to enable index merge join. - TiDBEnableIndexMergeJoin = "tidb_enable_index_merge_join" - - // TiDBTrackAggregateMemoryUsage indicates whether track the memory usage of aggregate function. - TiDBTrackAggregateMemoryUsage = "tidb_track_aggregate_memory_usage" - - // TiDBEnableExchangePartition indicates whether to enable exchange partition. - TiDBEnableExchangePartition = "tidb_enable_exchange_partition" - - // TiDBAllowFallbackToTiKV indicates the engine types whose unavailability triggers fallback to TiKV. - // Now we only support TiFlash. - TiDBAllowFallbackToTiKV = "tidb_allow_fallback_to_tikv" - - // TiDBEnableTopSQL indicates whether the top SQL is enabled. - TiDBEnableTopSQL = "tidb_enable_top_sql" - - // TiDBSourceID indicates the source ID of the TiDB server. - TiDBSourceID = "tidb_source_id" - - // TiDBTopSQLMaxTimeSeriesCount indicates the max number of statements been collected in each time series. - TiDBTopSQLMaxTimeSeriesCount = "tidb_top_sql_max_time_series_count" - - // TiDBTopSQLMaxMetaCount indicates the max capacity of the collect meta per second. - TiDBTopSQLMaxMetaCount = "tidb_top_sql_max_meta_count" - - // TiDBEnableLocalTxn indicates whether to enable Local Txn. - TiDBEnableLocalTxn = "tidb_enable_local_txn" - - // TiDBEnableMDL indicates whether to enable MDL. - TiDBEnableMDL = "tidb_enable_metadata_lock" - - // TiDBTSOClientBatchMaxWaitTime indicates the max value of the TSO Batch Wait interval time of PD client. - TiDBTSOClientBatchMaxWaitTime = "tidb_tso_client_batch_max_wait_time" - - // TiDBTxnCommitBatchSize is used to control the batch size of transaction commit related requests sent by TiDB to TiKV. - // If a single transaction has a large amount of writes, you can increase the batch size to improve the batch effect, - // setting too large will exceed TiKV's raft-entry-max-size limit and cause commit failure. - TiDBTxnCommitBatchSize = "tidb_txn_commit_batch_size" - - // TiDBEnableTSOFollowerProxy indicates whether to enable the TSO Follower Proxy feature of PD client. - TiDBEnableTSOFollowerProxy = "tidb_enable_tso_follower_proxy" - - // PDEnableFollowerHandleRegion indicates whether to enable the PD Follower handle region API. - PDEnableFollowerHandleRegion = "pd_enable_follower_handle_region" - - // TiDBEnableOrderedResultMode indicates if stabilize query results. - TiDBEnableOrderedResultMode = "tidb_enable_ordered_result_mode" - - // TiDBRemoveOrderbyInSubquery indicates whether to remove ORDER BY in subquery. - TiDBRemoveOrderbyInSubquery = "tidb_remove_orderby_in_subquery" - - // TiDBEnablePseudoForOutdatedStats indicates whether use pseudo for outdated stats - TiDBEnablePseudoForOutdatedStats = "tidb_enable_pseudo_for_outdated_stats" - - // TiDBRegardNULLAsPoint indicates whether regard NULL as point when optimizing - TiDBRegardNULLAsPoint = "tidb_regard_null_as_point" - - // TiDBTmpTableMaxSize indicates the max memory size of temporary tables. - TiDBTmpTableMaxSize = "tidb_tmp_table_max_size" - - // TiDBEnableLegacyInstanceScope indicates if instance scope can be set with SET SESSION. - TiDBEnableLegacyInstanceScope = "tidb_enable_legacy_instance_scope" - - // TiDBTableCacheLease indicates the read lock lease of a cached table. - TiDBTableCacheLease = "tidb_table_cache_lease" - - // TiDBStatsLoadSyncWait indicates the time sql execution will sync-wait for stats load. - TiDBStatsLoadSyncWait = "tidb_stats_load_sync_wait" - - // TiDBEnableMutationChecker indicates whether to check data consistency for mutations - TiDBEnableMutationChecker = "tidb_enable_mutation_checker" - // TiDBTxnAssertionLevel indicates how strict the assertion will be, which helps to detect and preventing data & - // index inconsistency problems. - TiDBTxnAssertionLevel = "tidb_txn_assertion_level" - - // TiDBIgnorePreparedCacheCloseStmt indicates whether to ignore close-stmt commands for prepared statements. - TiDBIgnorePreparedCacheCloseStmt = "tidb_ignore_prepared_cache_close_stmt" - - // TiDBEnableNewCostInterface is a internal switch to indicates whether to use the new cost calculation interface. - TiDBEnableNewCostInterface = "tidb_enable_new_cost_interface" - - // TiDBCostModelVersion is a internal switch to indicates the cost model version. - TiDBCostModelVersion = "tidb_cost_model_version" - - // TiDBIndexJoinDoubleReadPenaltyCostRate indicates whether to add some penalty cost to IndexJoin and how much of it. - // IndexJoin can cause plenty of extra double read tasks, which consume lots of resources and take a long time. - // Since the number of double read tasks is hard to estimated accurately, we leave this variable to let us can adjust this - // part of cost manually. - TiDBIndexJoinDoubleReadPenaltyCostRate = "tidb_index_join_double_read_penalty_cost_rate" - - // TiDBBatchPendingTiFlashCount indicates the maximum count of non-available TiFlash tables. - TiDBBatchPendingTiFlashCount = "tidb_batch_pending_tiflash_count" - - // TiDBQueryLogMaxLen is used to set the max length of the query in the log. - TiDBQueryLogMaxLen = "tidb_query_log_max_len" - - // TiDBEnableNoopVariables is used to indicate if noops appear in SHOW [GLOBAL] VARIABLES - TiDBEnableNoopVariables = "tidb_enable_noop_variables" - - // TiDBNonTransactionalIgnoreError is used to ignore error in non-transactional DMLs. - // When set to false, a non-transactional DML returns when it meets the first error. - // When set to true, a non-transactional DML finishes all batches even if errors are met in some batches. - TiDBNonTransactionalIgnoreError = "tidb_nontransactional_ignore_error" - - // Fine grained shuffle is disabled when TiFlashFineGrainedShuffleStreamCount is zero. - TiFlashFineGrainedShuffleStreamCount = "tiflash_fine_grained_shuffle_stream_count" - TiFlashFineGrainedShuffleBatchSize = "tiflash_fine_grained_shuffle_batch_size" - - // TiDBSimplifiedMetrics controls whether to unregister some unused metrics. - TiDBSimplifiedMetrics = "tidb_simplified_metrics" - - // TiDBMemoryDebugModeMinHeapInUse is used to set tidb memory debug mode trigger threshold. - // When set to 0, the function is disabled. - // When set to a negative integer, use memory debug mode to detect the issue of frequent allocation and release of memory. - // We do not actively trigger gc, and check whether the `tracker memory * (1+bias ratio) > heap in use` each 5s. - // When set to a positive integer, use memory debug mode to detect the issue of memory tracking inaccurate. - // We trigger runtime.GC() each 5s, and check whether the `tracker memory * (1+bias ratio) > heap in use`. - TiDBMemoryDebugModeMinHeapInUse = "tidb_memory_debug_mode_min_heap_inuse" - // TiDBMemoryDebugModeAlarmRatio is used set tidb memory debug mode bias ratio. Treat memory bias less than this ratio as noise. - TiDBMemoryDebugModeAlarmRatio = "tidb_memory_debug_mode_alarm_ratio" - - // TiDBEnableAnalyzeSnapshot indicates whether to read data on snapshot when collecting statistics. - // When set to false, ANALYZE reads the latest data. - // When set to true, ANALYZE reads data on the snapshot at the beginning of ANALYZE. - TiDBEnableAnalyzeSnapshot = "tidb_enable_analyze_snapshot" - - // TiDBDefaultStrMatchSelectivity controls some special cardinality estimation strategy for string match functions (like and regexp). - // When set to 0, Selectivity() will try to evaluate those functions with TopN and NULL in the stats to estimate, - // and the default selectivity and the selectivity for the histogram part will be 0.1. - // When set to (0, 1], Selectivity() will use the value of this variable as the default selectivity of those - // functions instead of the selectionFactor (0.8). - TiDBDefaultStrMatchSelectivity = "tidb_default_string_match_selectivity" - - // TiDBEnablePrepPlanCache indicates whether to enable prepared plan cache - TiDBEnablePrepPlanCache = "tidb_enable_prepared_plan_cache" - // TiDBPrepPlanCacheSize indicates the number of cached statements. - // This variable is deprecated, use tidb_session_plan_cache_size instead. - TiDBPrepPlanCacheSize = "tidb_prepared_plan_cache_size" - // TiDBEnablePrepPlanCacheMemoryMonitor indicates whether to enable prepared plan cache monitor - TiDBEnablePrepPlanCacheMemoryMonitor = "tidb_enable_prepared_plan_cache_memory_monitor" - - // TiDBEnableNonPreparedPlanCache indicates whether to enable non-prepared plan cache. - TiDBEnableNonPreparedPlanCache = "tidb_enable_non_prepared_plan_cache" - // TiDBEnableNonPreparedPlanCacheForDML indicates whether to enable non-prepared plan cache for DML statements. - TiDBEnableNonPreparedPlanCacheForDML = "tidb_enable_non_prepared_plan_cache_for_dml" - // TiDBNonPreparedPlanCacheSize controls the size of non-prepared plan cache. - // This variable is deprecated, use tidb_session_plan_cache_size instead. - TiDBNonPreparedPlanCacheSize = "tidb_non_prepared_plan_cache_size" - // TiDBPlanCacheMaxPlanSize controls the maximum size of a plan that can be cached. - TiDBPlanCacheMaxPlanSize = "tidb_plan_cache_max_plan_size" - // TiDBPlanCacheInvalidationOnFreshStats controls if plan cache will be invalidated automatically when - // related stats are analyzed after the plan cache is generated. - TiDBPlanCacheInvalidationOnFreshStats = "tidb_plan_cache_invalidation_on_fresh_stats" - // TiDBSessionPlanCacheSize controls the size of session plan cache. - TiDBSessionPlanCacheSize = "tidb_session_plan_cache_size" - - // TiDBEnableInstancePlanCache indicates whether to enable instance plan cache. - // If this variable is false, session-level plan cache will be used. - TiDBEnableInstancePlanCache = "tidb_enable_instance_plan_cache" - // TiDBInstancePlanCacheReservedPercentage indicates the percentage memory to evict. - TiDBInstancePlanCacheReservedPercentage = "tidb_instance_plan_cache_reserved_percentage" - // TiDBInstancePlanCacheMaxMemSize indicates the maximum memory size of instance plan cache. - TiDBInstancePlanCacheMaxMemSize = "tidb_instance_plan_cache_max_size" - - // TiDBConstraintCheckInPlacePessimistic controls whether to skip certain kinds of pessimistic locks. - TiDBConstraintCheckInPlacePessimistic = "tidb_constraint_check_in_place_pessimistic" - - // TiDBEnableForeignKey indicates whether to enable foreign key feature. - // TODO(crazycs520): remove this after foreign key GA. - TiDBEnableForeignKey = "tidb_enable_foreign_key" - - // TiDBOptRangeMaxSize is the max memory limit for ranges. When the optimizer estimates that the memory usage of complete - // ranges would exceed the limit, it chooses less accurate ranges such as full range. 0 indicates that there is no memory - // limit for ranges. - TiDBOptRangeMaxSize = "tidb_opt_range_max_size" - - // TiDBOptAdvancedJoinHint indicates whether the join method hint is compatible with join order hint. - TiDBOptAdvancedJoinHint = "tidb_opt_advanced_join_hint" - // TiDBOptUseInvisibleIndexes indicates whether to use invisible indexes. - TiDBOptUseInvisibleIndexes = "tidb_opt_use_invisible_indexes" - // TiDBAnalyzePartitionConcurrency is the number of concurrent workers to save statistics to the system tables. - TiDBAnalyzePartitionConcurrency = "tidb_analyze_partition_concurrency" - // TiDBMergePartitionStatsConcurrency indicates the concurrency when merge partition stats into global stats - TiDBMergePartitionStatsConcurrency = "tidb_merge_partition_stats_concurrency" - // TiDBEnableAsyncMergeGlobalStats indicates whether to enable async merge global stats - TiDBEnableAsyncMergeGlobalStats = "tidb_enable_async_merge_global_stats" - // TiDBOptPrefixIndexSingleScan indicates whether to do some optimizations to avoid double scan for prefix index. - // When set to true, `col is (not) null`(`col` is index prefix column) is regarded as index filter rather than table filter. - TiDBOptPrefixIndexSingleScan = "tidb_opt_prefix_index_single_scan" - - // TiDBEnableExternalTSRead indicates whether to enable read through an external ts - TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" - - // TiDBEnablePlanReplayerCapture indicates whether to enable plan replayer capture - TiDBEnablePlanReplayerCapture = "tidb_enable_plan_replayer_capture" - - // TiDBEnablePlanReplayerContinuousCapture indicates whether to enable continuous capture - TiDBEnablePlanReplayerContinuousCapture = "tidb_enable_plan_replayer_continuous_capture" - // TiDBEnableReusechunk indicates whether to enable chunk alloc - TiDBEnableReusechunk = "tidb_enable_reuse_chunk" - - // TiDBStoreBatchSize indicates the batch size of coprocessor in the same store. - TiDBStoreBatchSize = "tidb_store_batch_size" - - // MppExchangeCompressionMode indicates the data compression method in mpp exchange operator - MppExchangeCompressionMode = "mpp_exchange_compression_mode" - - // MppVersion indicates the mpp-version used to build mpp plan - MppVersion = "mpp_version" - - // TiDBPessimisticTransactionFairLocking controls whether fair locking for pessimistic transaction - // is enabled. - TiDBPessimisticTransactionFairLocking = "tidb_pessimistic_txn_fair_locking" - - // TiDBEnablePlanCacheForParamLimit controls whether prepare statement with parameterized limit can be cached - TiDBEnablePlanCacheForParamLimit = "tidb_enable_plan_cache_for_param_limit" - - // TiDBEnableINLJoinInnerMultiPattern indicates whether enable multi pattern for inner side of inl join - TiDBEnableINLJoinInnerMultiPattern = "tidb_enable_inl_join_inner_multi_pattern" - - // TiFlashComputeDispatchPolicy indicates how to dispatch task to tiflash_compute nodes. - TiFlashComputeDispatchPolicy = "tiflash_compute_dispatch_policy" - - // TiDBEnablePlanCacheForSubquery controls whether prepare statement with subquery can be cached - TiDBEnablePlanCacheForSubquery = "tidb_enable_plan_cache_for_subquery" - - // TiDBOptEnableLateMaterialization indicates whether to enable late materialization - TiDBOptEnableLateMaterialization = "tidb_opt_enable_late_materialization" - // TiDBLoadBasedReplicaReadThreshold is the wait duration threshold to enable replica read automatically. - TiDBLoadBasedReplicaReadThreshold = "tidb_load_based_replica_read_threshold" - - // TiDBOptOrderingIdxSelThresh is the threshold for optimizer to consider the ordering index. - TiDBOptOrderingIdxSelThresh = "tidb_opt_ordering_index_selectivity_threshold" - - // TiDBOptOrderingIdxSelRatio is the ratio the optimizer will assume applies when non indexed filtering rows are found - // via the ordering index. - TiDBOptOrderingIdxSelRatio = "tidb_opt_ordering_index_selectivity_ratio" - - // TiDBOptEnableMPPSharedCTEExecution indicates whether the optimizer try to build shared CTE scan during MPP execution. - TiDBOptEnableMPPSharedCTEExecution = "tidb_opt_enable_mpp_shared_cte_execution" - // TiDBOptFixControl makes the user able to control some details of the optimizer behavior. - TiDBOptFixControl = "tidb_opt_fix_control" - - // TiFlashReplicaRead is used to set the policy of TiFlash replica read when the query needs the TiFlash engine. - TiFlashReplicaRead = "tiflash_replica_read" - - // TiDBLockUnchangedKeys indicates whether to lock duplicate keys in INSERT IGNORE and REPLACE statements, - // or unchanged unique keys in UPDATE statements, see PR #42210 and #42713 - TiDBLockUnchangedKeys = "tidb_lock_unchanged_keys" - - // TiDBFastCheckTable enables fast check table. - TiDBFastCheckTable = "tidb_enable_fast_table_check" - - // TiDBAnalyzeSkipColumnTypes indicates the column types whose statistics would not be collected when executing the ANALYZE command. - TiDBAnalyzeSkipColumnTypes = "tidb_analyze_skip_column_types" - - // TiDBEnableCheckConstraint indicates whether to enable check constraint feature. - TiDBEnableCheckConstraint = "tidb_enable_check_constraint" - - // TiDBOptEnableHashJoin indicates whether to enable hash join. - TiDBOptEnableHashJoin = "tidb_opt_enable_hash_join" - - // TiDBHashJoinVersion indicates whether to use hash join implementation v2. - TiDBHashJoinVersion = "tidb_hash_join_version" - - // TiDBOptObjective indicates whether the optimizer should be more stable, predictable or more aggressive. - // Please see comments of SessionVars.OptObjective for details. - TiDBOptObjective = "tidb_opt_objective" - - // TiDBEnableParallelHashaggSpill is the name of the `tidb_enable_parallel_hashagg_spill` system variable - TiDBEnableParallelHashaggSpill = "tidb_enable_parallel_hashagg_spill" - - // TiDBTxnEntrySizeLimit indicates the max size of a entry in membuf. - TiDBTxnEntrySizeLimit = "tidb_txn_entry_size_limit" - - // TiDBSchemaCacheSize indicates the size of infoschema meta data which are cached in V2 implementation. - TiDBSchemaCacheSize = "tidb_schema_cache_size" - - // DivPrecisionIncrement indicates the number of digits by which to increase the scale of the result of - // division operations performed with the / operator. - DivPrecisionIncrement = "div_precision_increment" - - // TiDBEnableSharedLockPromotion indicates whether the `select for share` statement would be executed - // as `select for update` statements which do acquire pessimistic locks. - TiDBEnableSharedLockPromotion = "tidb_enable_shared_lock_promotion" - - // TiDBAccelerateUserCreationUpdate decides whether tidb will load & update the whole user's data in-memory. - TiDBAccelerateUserCreationUpdate = "tidb_accelerate_user_creation_update" -) - -// TiDB vars that have only global scope -const ( - // TiDBGCEnable turns garbage collection on or OFF - TiDBGCEnable = "tidb_gc_enable" - // TiDBGCRunInterval sets the interval that GC runs - TiDBGCRunInterval = "tidb_gc_run_interval" - // TiDBGCLifetime sets the retention window of older versions - TiDBGCLifetime = "tidb_gc_life_time" - // TiDBGCConcurrency sets the concurrency of garbage collection. -1 = AUTO value - TiDBGCConcurrency = "tidb_gc_concurrency" - // TiDBGCScanLockMode enables the green GC feature (deprecated) - TiDBGCScanLockMode = "tidb_gc_scan_lock_mode" - // TiDBGCMaxWaitTime sets max time for gc advances the safepoint delayed by active transactions - TiDBGCMaxWaitTime = "tidb_gc_max_wait_time" - // TiDBEnableEnhancedSecurity restricts SUPER users from certain operations. - TiDBEnableEnhancedSecurity = "tidb_enable_enhanced_security" - // TiDBEnableHistoricalStats enables the historical statistics feature (default off) - TiDBEnableHistoricalStats = "tidb_enable_historical_stats" - // TiDBPersistAnalyzeOptions persists analyze options for later analyze and auto-analyze - TiDBPersistAnalyzeOptions = "tidb_persist_analyze_options" - // TiDBEnableColumnTracking enables collecting predicate columns. - // DEPRECATED: This variable is deprecated, please do not use this variable. - TiDBEnableColumnTracking = "tidb_enable_column_tracking" - // TiDBAnalyzeColumnOptions specifies the default column selection strategy for both manual and automatic analyze operations. - // It accepts two values: - // `PREDICATE`: Analyze only the columns that are used in the predicates of the query. - // `ALL`: Analyze all columns in the table. - TiDBAnalyzeColumnOptions = "tidb_analyze_column_options" - // TiDBDisableColumnTrackingTime records the last time TiDBEnableColumnTracking is set off. - // It is used to invalidate the collected predicate columns after turning off TiDBEnableColumnTracking, which avoids physical deletion. - // It doesn't have cache in memory, and we directly get/set the variable value from/to mysql.tidb. - // DEPRECATED: This variable is deprecated, please do not use this variable. - TiDBDisableColumnTrackingTime = "tidb_disable_column_tracking_time" - // TiDBStatsLoadPseudoTimeout indicates whether to fallback to pseudo stats after load timeout. - TiDBStatsLoadPseudoTimeout = "tidb_stats_load_pseudo_timeout" - // TiDBMemQuotaBindingCache indicates the memory quota for the bind cache. - TiDBMemQuotaBindingCache = "tidb_mem_quota_binding_cache" - // TiDBRCReadCheckTS indicates the tso optimization for read-consistency read is enabled. - TiDBRCReadCheckTS = "tidb_rc_read_check_ts" - // TiDBRCWriteCheckTs indicates whether some special write statements don't get latest tso from PD at RC - TiDBRCWriteCheckTs = "tidb_rc_write_check_ts" - // TiDBCommitterConcurrency controls the number of running concurrent requests in the commit phase. - TiDBCommitterConcurrency = "tidb_committer_concurrency" - // TiDBEnableBatchDML enables batch dml. - TiDBEnableBatchDML = "tidb_enable_batch_dml" - // TiDBStatsCacheMemQuota records stats cache quota - TiDBStatsCacheMemQuota = "tidb_stats_cache_mem_quota" - // TiDBMemQuotaAnalyze indicates the memory quota for all analyze jobs. - TiDBMemQuotaAnalyze = "tidb_mem_quota_analyze" - // TiDBEnableAutoAnalyze determines whether TiDB executes automatic analysis. - // In test, we disable it by default. See GlobalSystemVariableInitialValue for details. - TiDBEnableAutoAnalyze = "tidb_enable_auto_analyze" - // TiDBEnableAutoAnalyzePriorityQueue determines whether TiDB executes automatic analysis with priority queue. - TiDBEnableAutoAnalyzePriorityQueue = "tidb_enable_auto_analyze_priority_queue" - // TiDBMemOOMAction indicates what operation TiDB perform when a single SQL statement exceeds - // the memory quota specified by tidb_mem_quota_query and cannot be spilled to disk. - TiDBMemOOMAction = "tidb_mem_oom_action" - // TiDBPrepPlanCacheMemoryGuardRatio is used to prevent [performance.max-memory] from being exceeded - TiDBPrepPlanCacheMemoryGuardRatio = "tidb_prepared_plan_cache_memory_guard_ratio" - // TiDBMaxAutoAnalyzeTime is the max time that auto analyze can run. If auto analyze runs longer than the value, it - // will be killed. 0 indicates that there is no time limit. - TiDBMaxAutoAnalyzeTime = "tidb_max_auto_analyze_time" - // TiDBAutoAnalyzeConcurrency is the concurrency of the auto analyze - TiDBAutoAnalyzeConcurrency = "tidb_auto_analyze_concurrency" - // TiDBEnableDistTask indicates whether to enable the distributed execute background tasks(For example DDL, Import etc). - TiDBEnableDistTask = "tidb_enable_dist_task" - // TiDBEnableFastCreateTable indicates whether to enable the fast create table feature. - TiDBEnableFastCreateTable = "tidb_enable_fast_create_table" - // TiDBGenerateBinaryPlan indicates whether binary plan should be generated in slow log and statements summary. - TiDBGenerateBinaryPlan = "tidb_generate_binary_plan" - // TiDBEnableGCAwareMemoryTrack indicates whether to turn-on GC-aware memory track. - TiDBEnableGCAwareMemoryTrack = "tidb_enable_gc_aware_memory_track" - // TiDBEnableTmpStorageOnOOM controls whether to enable the temporary storage for some operators - // when a single SQL statement exceeds the memory quota specified by the memory quota. - TiDBEnableTmpStorageOnOOM = "tidb_enable_tmp_storage_on_oom" - // TiDBDDLEnableFastReorg indicates whether to use lighting backfill process for adding index. - TiDBDDLEnableFastReorg = "tidb_ddl_enable_fast_reorg" - // TiDBDDLDiskQuota used to set disk quota for lightning add index. - TiDBDDLDiskQuota = "tidb_ddl_disk_quota" - // TiDBCloudStorageURI used to set a cloud storage uri for ddl add index and import into. - TiDBCloudStorageURI = "tidb_cloud_storage_uri" - // TiDBAutoBuildStatsConcurrency is the number of concurrent workers to automatically analyze tables or partitions. - // It is very similar to the `tidb_build_stats_concurrency` variable, but it is used for the auto analyze feature. - TiDBAutoBuildStatsConcurrency = "tidb_auto_build_stats_concurrency" - // TiDBSysProcScanConcurrency is used to set the scan concurrency of for backend system processes, like auto-analyze. - // For now, it controls the number of concurrent workers to scan regions to collect statistics (FMSketch, Samples). - TiDBSysProcScanConcurrency = "tidb_sysproc_scan_concurrency" - // TiDBServerMemoryLimit indicates the memory limit of the tidb-server instance. - TiDBServerMemoryLimit = "tidb_server_memory_limit" - // TiDBServerMemoryLimitSessMinSize indicates the minimal memory used of a session, that becomes a candidate for session kill. - TiDBServerMemoryLimitSessMinSize = "tidb_server_memory_limit_sess_min_size" - // TiDBServerMemoryLimitGCTrigger indicates the gc percentage of the TiDBServerMemoryLimit. - TiDBServerMemoryLimitGCTrigger = "tidb_server_memory_limit_gc_trigger" - // TiDBEnableGOGCTuner is to enable GOGC tuner. it can tuner GOGC - TiDBEnableGOGCTuner = "tidb_enable_gogc_tuner" - // TiDBGOGCTunerThreshold is to control the threshold of GOGC tuner. - TiDBGOGCTunerThreshold = "tidb_gogc_tuner_threshold" - // TiDBGOGCTunerMaxValue is the max value of GOGC that GOGC tuner can change to. - TiDBGOGCTunerMaxValue = "tidb_gogc_tuner_max_value" - // TiDBGOGCTunerMinValue is the min value of GOGC that GOGC tuner can change to. - TiDBGOGCTunerMinValue = "tidb_gogc_tuner_min_value" - // TiDBExternalTS is the ts to read through when the `TiDBEnableExternalTsRead` is on - TiDBExternalTS = "tidb_external_ts" - // TiDBTTLJobEnable is used to enable/disable scheduling ttl job - TiDBTTLJobEnable = "tidb_ttl_job_enable" - // TiDBTTLScanBatchSize is used to control the batch size in the SELECT statement for TTL jobs - TiDBTTLScanBatchSize = "tidb_ttl_scan_batch_size" - // TiDBTTLDeleteBatchSize is used to control the batch size in the DELETE statement for TTL jobs - TiDBTTLDeleteBatchSize = "tidb_ttl_delete_batch_size" - // TiDBTTLDeleteRateLimit is used to control the delete rate limit for TTL jobs in each node - TiDBTTLDeleteRateLimit = "tidb_ttl_delete_rate_limit" - // TiDBTTLJobScheduleWindowStartTime is used to restrict the start time of the time window of scheduling the ttl jobs. - TiDBTTLJobScheduleWindowStartTime = "tidb_ttl_job_schedule_window_start_time" - // TiDBTTLJobScheduleWindowEndTime is used to restrict the end time of the time window of scheduling the ttl jobs. - TiDBTTLJobScheduleWindowEndTime = "tidb_ttl_job_schedule_window_end_time" - // TiDBTTLScanWorkerCount indicates the count of the scan workers in each TiDB node - TiDBTTLScanWorkerCount = "tidb_ttl_scan_worker_count" - // TiDBTTLDeleteWorkerCount indicates the count of the delete workers in each TiDB node - TiDBTTLDeleteWorkerCount = "tidb_ttl_delete_worker_count" - // PasswordReuseHistory limit a few passwords to reuse. - PasswordReuseHistory = "password_history" - // PasswordReuseTime limit how long passwords can be reused. - PasswordReuseTime = "password_reuse_interval" - // TiDBHistoricalStatsDuration indicates the duration to remain tidb historical stats - TiDBHistoricalStatsDuration = "tidb_historical_stats_duration" - // TiDBEnableHistoricalStatsForCapture indicates whether use historical stats in plan replayer capture - TiDBEnableHistoricalStatsForCapture = "tidb_enable_historical_stats_for_capture" - // TiDBEnableResourceControl indicates whether resource control feature is enabled - TiDBEnableResourceControl = "tidb_enable_resource_control" - // TiDBResourceControlStrictMode indicates whether resource control strict mode is enabled. - // When strict mode is enabled, user need certain privilege to change session or statement resource group. - TiDBResourceControlStrictMode = "tidb_resource_control_strict_mode" - // TiDBStmtSummaryEnablePersistent indicates whether to enable file persistence for stmtsummary. - TiDBStmtSummaryEnablePersistent = "tidb_stmt_summary_enable_persistent" - // TiDBStmtSummaryFilename indicates the file name written by stmtsummary. - TiDBStmtSummaryFilename = "tidb_stmt_summary_filename" - // TiDBStmtSummaryFileMaxDays indicates how many days the files written by stmtsummary will be kept. - TiDBStmtSummaryFileMaxDays = "tidb_stmt_summary_file_max_days" - // TiDBStmtSummaryFileMaxSize indicates the maximum size (in mb) of a single file written by stmtsummary. - TiDBStmtSummaryFileMaxSize = "tidb_stmt_summary_file_max_size" - // TiDBStmtSummaryFileMaxBackups indicates the maximum number of files written by stmtsummary. - TiDBStmtSummaryFileMaxBackups = "tidb_stmt_summary_file_max_backups" - // TiDBTTLRunningTasks limits the count of running ttl tasks. Default to 0, means 3 times the count of TiKV (or no - // limitation, if the storage is not TiKV). - TiDBTTLRunningTasks = "tidb_ttl_running_tasks" - // AuthenticationLDAPSASLAuthMethodName defines the authentication method used by LDAP SASL authentication plugin - AuthenticationLDAPSASLAuthMethodName = "authentication_ldap_sasl_auth_method_name" - // AuthenticationLDAPSASLCAPath defines the ca certificate to verify LDAP connection in LDAP SASL authentication plugin - AuthenticationLDAPSASLCAPath = "authentication_ldap_sasl_ca_path" - // AuthenticationLDAPSASLTLS defines whether to use TLS connection in LDAP SASL authentication plugin - AuthenticationLDAPSASLTLS = "authentication_ldap_sasl_tls" - // AuthenticationLDAPSASLServerHost defines the server host of LDAP server for LDAP SASL authentication plugin - AuthenticationLDAPSASLServerHost = "authentication_ldap_sasl_server_host" - // AuthenticationLDAPSASLServerPort defines the port of LDAP server for LDAP SASL authentication plugin - AuthenticationLDAPSASLServerPort = "authentication_ldap_sasl_server_port" - // AuthenticationLDAPSASLReferral defines whether to enable LDAP referral for LDAP SASL authentication plugin - AuthenticationLDAPSASLReferral = "authentication_ldap_sasl_referral" - // AuthenticationLDAPSASLUserSearchAttr defines the attribute of username in LDAP server - AuthenticationLDAPSASLUserSearchAttr = "authentication_ldap_sasl_user_search_attr" - // AuthenticationLDAPSASLBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. - AuthenticationLDAPSASLBindBaseDN = "authentication_ldap_sasl_bind_base_dn" - // AuthenticationLDAPSASLBindRootDN defines the `dn` of the user to login the LDAP server and perform search. - AuthenticationLDAPSASLBindRootDN = "authentication_ldap_sasl_bind_root_dn" - // AuthenticationLDAPSASLBindRootPWD defines the password of the user to login the LDAP server and perform search. - AuthenticationLDAPSASLBindRootPWD = "authentication_ldap_sasl_bind_root_pwd" - // AuthenticationLDAPSASLInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSASLInitPoolSize = "authentication_ldap_sasl_init_pool_size" - // AuthenticationLDAPSASLMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSASLMaxPoolSize = "authentication_ldap_sasl_max_pool_size" - // AuthenticationLDAPSimpleAuthMethodName defines the authentication method used by LDAP Simple authentication plugin - AuthenticationLDAPSimpleAuthMethodName = "authentication_ldap_simple_auth_method_name" - // AuthenticationLDAPSimpleCAPath defines the ca certificate to verify LDAP connection in LDAP Simple authentication plugin - AuthenticationLDAPSimpleCAPath = "authentication_ldap_simple_ca_path" - // AuthenticationLDAPSimpleTLS defines whether to use TLS connection in LDAP Simple authentication plugin - AuthenticationLDAPSimpleTLS = "authentication_ldap_simple_tls" - // AuthenticationLDAPSimpleServerHost defines the server host of LDAP server for LDAP Simple authentication plugin - AuthenticationLDAPSimpleServerHost = "authentication_ldap_simple_server_host" - // AuthenticationLDAPSimpleServerPort defines the port of LDAP server for LDAP Simple authentication plugin - AuthenticationLDAPSimpleServerPort = "authentication_ldap_simple_server_port" - // AuthenticationLDAPSimpleReferral defines whether to enable LDAP referral for LDAP Simple authentication plugin - AuthenticationLDAPSimpleReferral = "authentication_ldap_simple_referral" - // AuthenticationLDAPSimpleUserSearchAttr defines the attribute of username in LDAP server - AuthenticationLDAPSimpleUserSearchAttr = "authentication_ldap_simple_user_search_attr" - // AuthenticationLDAPSimpleBindBaseDN defines the `dn` to search the users in. It's used to limit the search scope of TiDB. - AuthenticationLDAPSimpleBindBaseDN = "authentication_ldap_simple_bind_base_dn" - // AuthenticationLDAPSimpleBindRootDN defines the `dn` of the user to login the LDAP server and perform search. - AuthenticationLDAPSimpleBindRootDN = "authentication_ldap_simple_bind_root_dn" - // AuthenticationLDAPSimpleBindRootPWD defines the password of the user to login the LDAP server and perform search. - AuthenticationLDAPSimpleBindRootPWD = "authentication_ldap_simple_bind_root_pwd" - // AuthenticationLDAPSimpleInitPoolSize defines the init size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSimpleInitPoolSize = "authentication_ldap_simple_init_pool_size" - // AuthenticationLDAPSimpleMaxPoolSize defines the max size of connection pool to LDAP server for SASL plugin. - AuthenticationLDAPSimpleMaxPoolSize = "authentication_ldap_simple_max_pool_size" - // TiDBRuntimeFilterTypeName the value of is string, a runtime filter type list split by ",", such as: "IN,MIN_MAX" - TiDBRuntimeFilterTypeName = "tidb_runtime_filter_type" - // TiDBRuntimeFilterModeName the mode of runtime filter, such as "OFF", "LOCAL" - TiDBRuntimeFilterModeName = "tidb_runtime_filter_mode" - // TiDBSkipMissingPartitionStats controls how to handle missing partition stats when merging partition stats to global stats. - // When set to true, skip missing partition stats and continue to merge other partition stats to global stats. - // When set to false, give up merging partition stats to global stats. - TiDBSkipMissingPartitionStats = "tidb_skip_missing_partition_stats" - // TiDBSessionAlias indicates the alias of a session which is used for tracing. - TiDBSessionAlias = "tidb_session_alias" - // TiDBServiceScope indicates the role for tidb for distributed task framework. - TiDBServiceScope = "tidb_service_scope" - // TiDBSchemaVersionCacheLimit defines the capacity size of domain infoSchema cache. - TiDBSchemaVersionCacheLimit = "tidb_schema_version_cache_limit" - // TiDBEnableTiFlashPipelineMode means if we should use pipeline model to execute query or not in tiflash. - // It's deprecated and setting it will not have any effect. - TiDBEnableTiFlashPipelineMode = "tidb_enable_tiflash_pipeline_model" - // TiDBIdleTransactionTimeout indicates the maximum time duration a transaction could be idle, unit is second. - // Any idle transaction will be killed after being idle for `tidb_idle_transaction_timeout` seconds. - // This is similar to https://docs.percona.com/percona-server/5.7/management/innodb_kill_idle_trx.html and https://mariadb.com/kb/en/transaction-timeouts/ - TiDBIdleTransactionTimeout = "tidb_idle_transaction_timeout" - // TiDBLowResolutionTSOUpdateInterval defines how often to refresh low resolution timestamps. - TiDBLowResolutionTSOUpdateInterval = "tidb_low_resolution_tso_update_interval" - // TiDBDMLType indicates the execution type of DML in TiDB. - // The value can be STANDARD, BULK. - // Currently, the BULK mode only affects auto-committed DML. - TiDBDMLType = "tidb_dml_type" - // TiFlashHashAggPreAggMode indicates the policy of 1st hashagg. - TiFlashHashAggPreAggMode = "tiflash_hashagg_preaggregation_mode" - // TiDBEnableLazyCursorFetch defines whether to enable the lazy cursor fetch. If it's `OFF`, all results of - // of a cursor will be stored in the tidb node in `EXECUTE` command. - TiDBEnableLazyCursorFetch = "tidb_enable_lazy_cursor_fetch" - // TiDBTSOClientRPCMode controls how the TSO client performs the TSO RPC requests. It internally controls the - // concurrency of the RPC. This variable provides an approach to tune the latency of getting timestamps from PD. - TiDBTSOClientRPCMode = "tidb_tso_client_rpc_mode" - // TiDBCircuitBreakerPDMetadataErrorRateThresholdPct variable is used to set percent of errors to trip the circuit breaker for get region calls to PD - // https://github.com/tikv/rfcs/blob/master/text/0115-circuit-breaker.md - TiDBCircuitBreakerPDMetadataErrorRateThresholdPct = "tidb_cb_pd_metadata_error_rate_threshold_pct" -) - -// TiDB intentional limits, can be raised in the future. -const ( - // MaxConfigurableConcurrency is the maximum number of "threads" (goroutines) that can be specified - // for any type of configuration item that has concurrent workers. - MaxConfigurableConcurrency = 256 - - // MaxShardRowIDBits is the maximum number of bits that can be used for row-id sharding. - MaxShardRowIDBits = 15 - - // MaxPreSplitRegions is the maximum number of regions that can be pre-split. - MaxPreSplitRegions = 15 -) - -// Default TiDB system variable values. -const ( - DefHostname = "localhost" - DefIndexLookupConcurrency = ConcurrencyUnset - DefIndexLookupJoinConcurrency = ConcurrencyUnset - DefIndexSerialScanConcurrency = 1 - DefIndexJoinBatchSize = 25000 - DefIndexLookupSize = 20000 - DefDistSQLScanConcurrency = 15 - DefAnalyzeDistSQLScanConcurrency = 4 - DefBuildStatsConcurrency = 2 - DefBuildSamplingStatsConcurrency = 2 - DefAutoAnalyzeRatio = 0.5 - DefAutoAnalyzeStartTime = "00:00 +0000" - DefAutoAnalyzeEndTime = "23:59 +0000" - DefAutoIncrementIncrement = 1 - DefAutoIncrementOffset = 1 - DefChecksumTableConcurrency = 4 - DefSkipUTF8Check = false - DefSkipASCIICheck = false - DefOptAggPushDown = false - DefOptDeriveTopN = false - DefOptCartesianBCJ = 1 - DefOptMPPOuterJoinFixedBuildSide = false - DefOptWriteRowID = false - DefOptEnableCorrelationAdjustment = true - DefOptLimitPushDownThreshold = 100 - DefOptCorrelationThreshold = 0.9 - DefOptCorrelationExpFactor = 1 - DefOptCPUFactor = 3.0 - DefOptCopCPUFactor = 3.0 - DefOptTiFlashConcurrencyFactor = 24.0 - DefOptNetworkFactor = 1.0 - DefOptScanFactor = 1.5 - DefOptDescScanFactor = 3.0 - DefOptSeekFactor = 20.0 - DefOptMemoryFactor = 0.001 - DefOptDiskFactor = 1.5 - DefOptConcurrencyFactor = 3.0 - DefOptForceInlineCTE = false - DefOptInSubqToJoinAndAgg = true - DefOptPreferRangeScan = true - DefBatchInsert = false - DefBatchDelete = false - DefBatchCommit = false - DefCurretTS = 0 - DefInitChunkSize = 32 - DefMinPagingSize = int(paging.MinPagingSize) - DefMaxPagingSize = int(paging.MaxPagingSize) - DefMaxChunkSize = 1024 - DefDMLBatchSize = 0 - DefMaxPreparedStmtCount = -1 - DefWaitTimeout = 28800 - DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. - DefTiDBMemQuotaBindingCache = 64 << 20 // 64MB. - DefTiDBGeneralLog = false - DefTiDBPProfSQLCPU = 0 - DefTiDBRetryLimit = 10 - DefTiDBDisableTxnAutoRetry = true - DefTiDBConstraintCheckInPlace = false - DefTiDBHashJoinConcurrency = ConcurrencyUnset - DefTiDBProjectionConcurrency = ConcurrencyUnset - DefBroadcastJoinThresholdSize = 100 * 1024 * 1024 - DefBroadcastJoinThresholdCount = 10 * 1024 - DefPreferBCJByExchangeDataSize = false - DefTiDBOptimizerSelectivityLevel = 0 - DefTiDBOptimizerEnableNewOFGB = false - DefTiDBEnableOuterJoinReorder = true - DefTiDBEnableNAAJ = true - DefTiDBAllowBatchCop = 1 - DefShardRowIDBits = 0 - DefPreSplitRegions = 0 - DefBlockEncryptionMode = "aes-128-ecb" - DefTiDBAllowMPPExecution = true - DefTiDBAllowTiFlashCop = false - DefTiDBHashExchangeWithNewCollation = true - DefTiDBEnforceMPPExecution = false - DefTiFlashMaxThreads = -1 - DefTiFlashMaxBytesBeforeExternalJoin = -1 - DefTiFlashMaxBytesBeforeExternalGroupBy = -1 - DefTiFlashMaxBytesBeforeExternalSort = -1 - DefTiFlashMemQuotaQueryPerNode = 0 - DefTiFlashQuerySpillRatio = 0.7 - DefTiDBEnableTiFlashPipelineMode = true - DefTiDBMPPStoreFailTTL = "60s" - DefTiDBTxnMode = PessimisticTxnMode - DefTiDBRowFormatV1 = 1 - DefTiDBRowFormatV2 = 2 - DefTiDBDDLReorgWorkerCount = 4 - DefTiDBDDLReorgBatchSize = 256 - DefTiDBDDLFlashbackConcurrency = 64 - DefTiDBDDLErrorCountLimit = 512 - DefTiDBDDLReorgMaxWriteSpeed = 0 - DefTiDBMaxDeltaSchemaCount = 1024 - DefTiDBPlacementMode = PlacementModeStrict - DefTiDBEnableAutoIncrementInGenerated = false - DefTiDBHashAggPartialConcurrency = ConcurrencyUnset - DefTiDBHashAggFinalConcurrency = ConcurrencyUnset - DefTiDBWindowConcurrency = ConcurrencyUnset - DefTiDBMergeJoinConcurrency = 1 // disable optimization by default - DefTiDBStreamAggConcurrency = 1 - DefTiDBForcePriority = mysql.NoPriority - DefEnableWindowFunction = true - DefEnablePipelinedWindowFunction = true - DefEnableStrictDoubleTypeCheck = true - DefEnableVectorizedExpression = true - DefTiDBOptJoinReorderThreshold = 0 - DefTiDBDDLSlowOprThreshold = 300 - DefTiDBUseFastAnalyze = false - DefTiDBSkipIsolationLevelCheck = false - DefTiDBExpensiveQueryTimeThreshold = 60 // 60s - DefTiDBExpensiveTxnTimeThreshold = 60 * 10 // 10 minutes - DefTiDBScatterRegion = ScatterOff - DefTiDBWaitSplitRegionFinish = true - DefWaitSplitRegionTimeout = 300 // 300s - DefTiDBEnableNoopFuncs = Off - DefTiDBEnableNoopVariables = true - DefTiDBAllowRemoveAutoInc = false - DefTiDBUsePlanBaselines = true - DefTiDBEvolvePlanBaselines = false - DefTiDBEvolvePlanTaskMaxTime = 600 // 600s - DefTiDBEvolvePlanTaskStartTime = "00:00 +0000" - DefTiDBEvolvePlanTaskEndTime = "23:59 +0000" - DefInnodbLockWaitTimeout = 50 // 50s - DefTiDBStoreLimit = 0 - DefTiDBMetricSchemaStep = 60 // 60s - DefTiDBMetricSchemaRangeDuration = 60 // 60s - DefTiDBFoundInPlanCache = false - DefTiDBFoundInBinding = false - DefTiDBEnableCollectExecutionInfo = true - DefTiDBAllowAutoRandExplicitInsert = false - DefTiDBEnableClusteredIndex = ClusteredIndexDefModeOn - DefTiDBRedactLog = Off - DefTiDBRestrictedReadOnly = false - DefTiDBSuperReadOnly = false - DefTiDBShardAllocateStep = math.MaxInt64 - DefTiDBEnableTelemetry = false - DefTiDBEnableParallelApply = false - DefTiDBPartitionPruneMode = "dynamic" - DefTiDBEnableRateLimitAction = false - DefTiDBEnableAsyncCommit = false - DefTiDBEnable1PC = false - DefTiDBGuaranteeLinearizability = true - DefTiDBAnalyzeVersion = 2 - // Deprecated: This variable is deprecated, please do not use this variable. - DefTiDBAutoAnalyzePartitionBatchSize = mysql.PartitionCountLimit - DefTiDBEnableIndexMergeJoin = false - DefTiDBTrackAggregateMemoryUsage = true - DefCTEMaxRecursionDepth = 1000 - DefTiDBTmpTableMaxSize = 64 << 20 // 64MB. - DefTiDBEnableLocalTxn = false - DefTiDBTSOClientBatchMaxWaitTime = 0.0 // 0ms - DefTiDBEnableTSOFollowerProxy = false - DefPDEnableFollowerHandleRegion = true - DefTiDBEnableOrderedResultMode = false - DefTiDBEnablePseudoForOutdatedStats = false - DefTiDBRegardNULLAsPoint = true - DefEnablePlacementCheck = true - DefTimestamp = "0" - DefTimestampFloat = 0.0 - DefTiDBEnableStmtSummary = true - DefTiDBStmtSummaryInternalQuery = false - DefTiDBStmtSummaryRefreshInterval = 1800 - DefTiDBStmtSummaryHistorySize = 24 - DefTiDBStmtSummaryMaxStmtCount = 3000 - DefTiDBStmtSummaryMaxSQLLength = 4096 - DefTiDBCapturePlanBaseline = Off - DefTiDBIgnoreInlistPlanDigest = false - DefTiDBEnableIndexMerge = true - DefEnableLegacyInstanceScope = true - DefTiDBTableCacheLease = 3 // 3s - DefTiDBPersistAnalyzeOptions = true - DefTiDBStatsLoadSyncWait = 100 - DefTiDBStatsLoadPseudoTimeout = true - DefSysdateIsNow = false - DefTiDBEnableParallelHashaggSpill = true - DefTiDBEnableMutationChecker = false - DefTiDBTxnAssertionLevel = AssertionOffStr - DefTiDBIgnorePreparedCacheCloseStmt = false - DefTiDBBatchPendingTiFlashCount = 4000 - DefRCReadCheckTS = false - DefTiDBRemoveOrderbyInSubquery = true - DefTiDBSkewDistinctAgg = false - DefTiDB3StageDistinctAgg = true - DefTiDB3StageMultiDistinctAgg = false - DefTiDBOptExplainEvaledSubquery = false - DefTiDBReadStaleness = 0 - DefTiDBGCMaxWaitTime = 24 * 60 * 60 - DefMaxAllowedPacket uint64 = 67108864 - DefTiDBEnableBatchDML = false - DefTiDBMemQuotaQuery = memory.DefMemQuotaQuery // 1GB - DefTiDBStatsCacheMemQuota = 0 - MaxTiDBStatsCacheMemQuota = 1024 * 1024 * 1024 * 1024 // 1TB - DefTiDBQueryLogMaxLen = 4096 - DefRequireSecureTransport = false - DefTiDBCommitterConcurrency = 128 - DefTiDBBatchDMLIgnoreError = false - DefTiDBMemQuotaAnalyze = -1 - DefTiDBEnableAutoAnalyze = true - DefTiDBEnableAutoAnalyzePriorityQueue = true - DefTiDBAnalyzeColumnOptions = "PREDICATE" - DefTiDBMemOOMAction = "CANCEL" - DefTiDBMaxAutoAnalyzeTime = 12 * 60 * 60 - DefTiDBAutoAnalyzeConcurrency = 1 - DefTiDBEnablePrepPlanCache = true - DefTiDBPrepPlanCacheSize = 100 - DefTiDBSessionPlanCacheSize = 100 - DefTiDBEnablePrepPlanCacheMemoryMonitor = true - DefTiDBPrepPlanCacheMemoryGuardRatio = 0.1 - DefTiDBEnableWorkloadBasedLearning = false - DefTiDBWorkloadBasedLearningInterval = 24 * time.Hour - DefTiDBEnableDistTask = true - DefTiDBEnableFastCreateTable = true - DefTiDBSimplifiedMetrics = false - DefTiDBEnablePaging = true - DefTiFlashFineGrainedShuffleStreamCount = 0 - DefStreamCountWhenMaxThreadsNotSet = 8 - DefTiFlashFineGrainedShuffleBatchSize = 8192 - DefAdaptiveClosestReadThreshold = 4096 - DefTiDBEnableAnalyzeSnapshot = false - DefTiDBGenerateBinaryPlan = true - DefEnableTiDBGCAwareMemoryTrack = false - DefTiDBDefaultStrMatchSelectivity = 0.8 - DefTiDBEnableTmpStorageOnOOM = true - DefTiDBEnableMDL = true - DefTiFlashFastScan = false - DefMemoryUsageAlarmRatio = 0.7 - DefMemoryUsageAlarmKeepRecordNum = 5 - DefTiDBEnableFastReorg = true - DefTiDBDDLDiskQuota = 100 * 1024 * 1024 * 1024 // 100GB - DefExecutorConcurrency = 5 - DefTiDBEnableNonPreparedPlanCache = false - DefTiDBEnableNonPreparedPlanCacheForDML = false - DefTiDBNonPreparedPlanCacheSize = 100 - DefTiDBPlanCacheMaxPlanSize = 2 * size.MB - DefTiDBInstancePlanCacheMaxMemSize = 100 * size.MB - MinTiDBInstancePlanCacheMemSize = 100 * size.MB - DefTiDBInstancePlanCacheReservedPercentage = 0.1 - // MaxDDLReorgBatchSize is exported for testing. - MaxDDLReorgBatchSize int32 = 10240 - MinDDLReorgBatchSize int32 = 32 - MinExpensiveQueryTimeThreshold uint64 = 10 // 10s - MinExpensiveTxnTimeThreshold uint64 = 60 // 60s - DefTiDBAutoBuildStatsConcurrency = 1 - DefTiDBSysProcScanConcurrency = 1 - DefTiDBRcWriteCheckTs = false - DefTiDBForeignKeyChecks = true - DefTiDBOptAdvancedJoinHint = true - DefTiDBAnalyzePartitionConcurrency = 2 - DefTiDBOptRangeMaxSize = 64 * int64(size.MB) // 64 MB - DefTiDBCostModelVer = 2 - DefTiDBServerMemoryLimitSessMinSize = 128 << 20 - DefTiDBMergePartitionStatsConcurrency = 1 - DefTiDBServerMemoryLimitGCTrigger = 0.7 - DefTiDBEnableGOGCTuner = true - // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. - DefTiDBGOGCTunerThreshold float64 = 0.6 - DefTiDBGOGCMaxValue = 500 - DefTiDBGOGCMinValue = 100 - DefTiDBOptPrefixIndexSingleScan = true - DefTiDBEnableAsyncMergeGlobalStats = true - DefTiDBExternalTS = 0 - DefTiDBEnableExternalTSRead = false - DefTiDBEnableReusechunk = true - DefTiDBUseAlloc = false - DefTiDBEnablePlanReplayerCapture = true - DefTiDBIndexMergeIntersectionConcurrency = ConcurrencyUnset - DefTiDBTTLJobEnable = true - DefTiDBTTLScanBatchSize = 500 - DefTiDBTTLScanBatchMaxSize = 10240 - DefTiDBTTLScanBatchMinSize = 1 - DefTiDBTTLDeleteBatchSize = 100 - DefTiDBTTLDeleteBatchMaxSize = 10240 - DefTiDBTTLDeleteBatchMinSize = 1 - DefTiDBTTLDeleteRateLimit = 0 - DefTiDBTTLRunningTasks = -1 - DefPasswordReuseHistory = 0 - DefPasswordReuseTime = 0 - DefTiDBStoreBatchSize = 4 - DefTiDBHistoricalStatsDuration = 7 * 24 * time.Hour - DefTiDBEnableHistoricalStatsForCapture = false - DefTiDBTTLJobScheduleWindowStartTime = "00:00 +0000" - DefTiDBTTLJobScheduleWindowEndTime = "23:59 +0000" - DefTiDBTTLScanWorkerCount = 4 - DefTiDBTTLDeleteWorkerCount = 4 - DefaultExchangeCompressionMode = ExchangeCompressionModeUnspecified - DefTiDBEnableResourceControl = true - DefTiDBResourceControlStrictMode = true - DefTiDBPessimisticTransactionFairLocking = false - DefTiDBEnablePlanCacheForParamLimit = true - DefTiDBEnableINLJoinMultiPattern = true - DefTiFlashComputeDispatchPolicy = DispatchPolicyConsistentHashStr - DefTiDBEnablePlanCacheForSubquery = true - DefTiDBLoadBasedReplicaReadThreshold = time.Second - DefTiDBOptEnableLateMaterialization = true - DefTiDBOptOrderingIdxSelThresh = 0.0 - DefTiDBOptOrderingIdxSelRatio = -1 - DefTiDBOptEnableMPPSharedCTEExecution = false - DefTiDBPlanCacheInvalidationOnFreshStats = true - DefTiDBEnableRowLevelChecksum = false - DefAuthenticationLDAPSASLAuthMethodName = "SCRAM-SHA-1" - DefAuthenticationLDAPSASLServerPort = 389 - DefAuthenticationLDAPSASLTLS = false - DefAuthenticationLDAPSASLUserSearchAttr = "uid" - DefAuthenticationLDAPSASLInitPoolSize = 10 - DefAuthenticationLDAPSASLMaxPoolSize = 1000 - DefAuthenticationLDAPSimpleAuthMethodName = "SIMPLE" - DefAuthenticationLDAPSimpleServerPort = 389 - DefAuthenticationLDAPSimpleTLS = false - DefAuthenticationLDAPSimpleUserSearchAttr = "uid" - DefAuthenticationLDAPSimpleInitPoolSize = 10 - DefAuthenticationLDAPSimpleMaxPoolSize = 1000 - DefTiFlashReplicaRead = AllReplicaStr - DefTiDBEnableFastCheckTable = true - DefRuntimeFilterType = "IN" - DefRuntimeFilterMode = "OFF" - DefTiDBLockUnchangedKeys = true - DefTiDBEnableCheckConstraint = false - DefTiDBSkipMissingPartitionStats = true - DefTiDBOptEnableHashJoin = true - DefTiDBHashJoinVersion = joinversion.HashJoinVersionOptimized - DefTiDBOptObjective = OptObjectiveModerate - DefTiDBSchemaVersionCacheLimit = 16 - DefTiDBIdleTransactionTimeout = 0 - DefTiDBTxnEntrySizeLimit = 0 - DefTiDBSchemaCacheSize = 512 * 1024 * 1024 - DefTiDBLowResolutionTSOUpdateInterval = 2000 - DefDivPrecisionIncrement = 4 - DefTiDBDMLType = "STANDARD" - DefGroupConcatMaxLen = uint64(1024) - DefDefaultWeekFormat = "0" - DefTiFlashPreAggMode = ForcePreAggStr - DefTiDBEnableLazyCursorFetch = false - DefOptEnableProjectionPushDown = true - DefTiDBEnableSharedLockPromotion = false - DefTiDBTSOClientRPCMode = TSOClientRPCModeDefault - DefTiDBCircuitBreakerPDMetaErrorRatePct = 0 - DefTiDBAccelerateUserCreationUpdate = false -) - -// Process global variables. -var ( - ProcessGeneralLog = atomic.NewBool(false) - RunAutoAnalyze = atomic.NewBool(DefTiDBEnableAutoAnalyze) - EnableAutoAnalyzePriorityQueue = atomic.NewBool(DefTiDBEnableAutoAnalyzePriorityQueue) - // AnalyzeColumnOptions is a global variable that indicates the default column choice for ANALYZE. - // The value of this variable is a string that can be one of the following values: - // "PREDICATE", "ALL". - // The behavior of the analyze operation depends on the value of `tidb_persist_analyze_options`: - // 1. If `tidb_persist_analyze_options` is enabled and the column choice from the analyze options record is set to `default`, - // the value of `tidb_analyze_column_options` determines the behavior of the analyze operation. - // 2. If `tidb_persist_analyze_options` is disabled, `tidb_analyze_column_options` is used directly to decide - // whether to analyze all columns or just the predicate columns. - AnalyzeColumnOptions = atomic.NewString(DefTiDBAnalyzeColumnOptions) - GlobalLogMaxDays = atomic.NewInt32(int32(config.GetGlobalConfig().Log.File.MaxDays)) - QueryLogMaxLen = atomic.NewInt32(DefTiDBQueryLogMaxLen) - EnablePProfSQLCPU = atomic.NewBool(false) - EnableBatchDML = atomic.NewBool(false) - EnableTmpStorageOnOOM = atomic.NewBool(DefTiDBEnableTmpStorageOnOOM) - DDLReorgWorkerCounter int32 = DefTiDBDDLReorgWorkerCount - DDLReorgBatchSize int32 = DefTiDBDDLReorgBatchSize - DDLFlashbackConcurrency int32 = DefTiDBDDLFlashbackConcurrency - DDLErrorCountLimit int64 = DefTiDBDDLErrorCountLimit - DDLReorgRowFormat int64 = DefTiDBRowFormatV2 - DDLReorgMaxWriteSpeed = atomic.NewInt64(DefTiDBDDLReorgMaxWriteSpeed) - MaxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount - // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. - DDLSlowOprThreshold = config.GetGlobalConfig().Instance.DDLSlowOprThreshold - ForcePriority = int32(DefTiDBForcePriority) - MaxOfMaxAllowedPacket uint64 = 1073741824 - ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold - ExpensiveTxnTimeThreshold uint64 = DefTiDBExpensiveTxnTimeThreshold - MemoryUsageAlarmRatio = atomic.NewFloat64(DefMemoryUsageAlarmRatio) - MemoryUsageAlarmKeepRecordNum = atomic.NewInt64(DefMemoryUsageAlarmKeepRecordNum) - EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn) - MaxTSOBatchWaitInterval = atomic.NewFloat64(DefTiDBTSOClientBatchMaxWaitTime) - EnableTSOFollowerProxy = atomic.NewBool(DefTiDBEnableTSOFollowerProxy) - EnablePDFollowerHandleRegion = atomic.NewBool(DefPDEnableFollowerHandleRegion) - RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly) - VarTiDBSuperReadOnly = atomic.NewBool(DefTiDBSuperReadOnly) - PersistAnalyzeOptions = atomic.NewBool(DefTiDBPersistAnalyzeOptions) - TableCacheLease = atomic.NewInt64(DefTiDBTableCacheLease) - StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) - StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) - MemQuotaBindingCache = atomic.NewInt64(DefTiDBMemQuotaBindingCache) - GCMaxWaitTime = atomic.NewInt64(DefTiDBGCMaxWaitTime) - StatsCacheMemQuota = atomic.NewInt64(DefTiDBStatsCacheMemQuota) - OOMAction = atomic.NewString(DefTiDBMemOOMAction) - MaxAutoAnalyzeTime = atomic.NewInt64(DefTiDBMaxAutoAnalyzeTime) - // variables for plan cache - PreparedPlanCacheMemoryGuardRatio = atomic.NewFloat64(DefTiDBPrepPlanCacheMemoryGuardRatio) - EnableInstancePlanCache = atomic.NewBool(false) - InstancePlanCacheReservedPercentage = atomic.NewFloat64(0.1) - InstancePlanCacheMaxMemSize = atomic.NewInt64(int64(DefTiDBInstancePlanCacheMaxMemSize)) - EnableDistTask = atomic.NewBool(DefTiDBEnableDistTask) - EnableFastCreateTable = atomic.NewBool(DefTiDBEnableFastCreateTable) - EnableNoopVariables = atomic.NewBool(DefTiDBEnableNoopVariables) - EnableMDL = atomic.NewBool(false) - AutoAnalyzePartitionBatchSize = atomic.NewInt64(DefTiDBAutoAnalyzePartitionBatchSize) - AutoAnalyzeConcurrency = atomic.NewInt32(DefTiDBAutoAnalyzeConcurrency) - // TODO: set value by session variable - EnableWorkloadBasedLearning = atomic.NewBool(DefTiDBEnableWorkloadBasedLearning) - WorkloadBasedLearningInterval = atomic.NewDuration(DefTiDBWorkloadBasedLearningInterval) - // EnableFastReorg indicates whether to use lightning to enhance DDL reorg performance. - EnableFastReorg = atomic.NewBool(DefTiDBEnableFastReorg) - // DDLDiskQuota is the temporary variable for set disk quota for lightning - DDLDiskQuota = atomic.NewUint64(DefTiDBDDLDiskQuota) - // EnableForeignKey indicates whether to enable foreign key feature. - EnableForeignKey = atomic.NewBool(true) - EnableRCReadCheckTS = atomic.NewBool(false) - // EnableRowLevelChecksum indicates whether to append checksum to row values. - EnableRowLevelChecksum = atomic.NewBool(DefTiDBEnableRowLevelChecksum) - LowResolutionTSOUpdateInterval = atomic.NewUint32(DefTiDBLowResolutionTSOUpdateInterval) - - // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). - // It should be a const and shouldn't be modified after tidb is started. - DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() - GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) - PasswordValidationLength = atomic.NewInt32(8) - PasswordValidationMixedCaseCount = atomic.NewInt32(1) - PasswordValidtaionNumberCount = atomic.NewInt32(1) - PasswordValidationSpecialCharCount = atomic.NewInt32(1) - EnableTTLJob = atomic.NewBool(DefTiDBTTLJobEnable) - TTLScanBatchSize = atomic.NewInt64(DefTiDBTTLScanBatchSize) - TTLDeleteBatchSize = atomic.NewInt64(DefTiDBTTLDeleteBatchSize) - TTLDeleteRateLimit = atomic.NewInt64(DefTiDBTTLDeleteRateLimit) - TTLJobScheduleWindowStartTime = atomic.NewTime( - mustParseTime( - FullDayTimeFormat, - DefTiDBTTLJobScheduleWindowStartTime, - ), - ) - TTLJobScheduleWindowEndTime = atomic.NewTime( - mustParseTime( - FullDayTimeFormat, - DefTiDBTTLJobScheduleWindowEndTime, - ), - ) - TTLScanWorkerCount = atomic.NewInt32(DefTiDBTTLScanWorkerCount) - TTLDeleteWorkerCount = atomic.NewInt32(DefTiDBTTLDeleteWorkerCount) - PasswordHistory = atomic.NewInt64(DefPasswordReuseHistory) - PasswordReuseInterval = atomic.NewInt64(DefPasswordReuseTime) - IsSandBoxModeEnabled = atomic.NewBool(false) - MaxPreparedStmtCountValue = atomic.NewInt64(DefMaxPreparedStmtCount) - HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration) - EnableHistoricalStatsForCapture = atomic.NewBool(DefTiDBEnableHistoricalStatsForCapture) - TTLRunningTasks = atomic.NewInt32(DefTiDBTTLRunningTasks) - // always set the default value to false because the resource control in kv-client is not inited - // It will be initialized to the right value after the first call of `rebuildSysVarCache` - EnableResourceControl = atomic.NewBool(false) - EnableResourceControlStrictMode = atomic.NewBool(true) - EnableCheckConstraint = atomic.NewBool(DefTiDBEnableCheckConstraint) - SkipMissingPartitionStats = atomic.NewBool(DefTiDBSkipMissingPartitionStats) - TiFlashEnablePipelineMode = atomic.NewBool(DefTiDBEnableTiFlashPipelineMode) - ServiceScope = atomic.NewString("") - SchemaVersionCacheLimit = atomic.NewInt64(DefTiDBSchemaVersionCacheLimit) - CloudStorageURI = atomic.NewString("") - IgnoreInlistPlanDigest = atomic.NewBool(DefTiDBIgnoreInlistPlanDigest) - TxnEntrySizeLimit = atomic.NewUint64(DefTiDBTxnEntrySizeLimit) - - SchemaCacheSize = atomic.NewUint64(DefTiDBSchemaCacheSize) - SchemaCacheSizeOriginText = atomic.NewString(strconv.Itoa(DefTiDBSchemaCacheSize)) - AccelerateUserCreationUpdate = atomic.NewBool(DefTiDBAccelerateUserCreationUpdate) -) - -func serverMemoryLimitDefaultValue() string { - total, err := memory.MemTotal() - if err == nil && total != 0 { - return "80%" - } - return "0" -} - -func mustParseDuration(str string) time.Duration { - duration, err := time.ParseDuration(str) - if err != nil { - panic(fmt.Sprintf("%s is not a duration", str)) - } - - return duration -} - -func mustParseTime(layout string, str string) time.Time { - time, err := time.ParseInLocation(layout, str, time.UTC) - if err != nil { - panic(fmt.Sprintf("%s is not in %s duration format", str, layout)) - } - - return time -} - -const ( - // OptObjectiveModerate is a possible value and the default value for TiDBOptObjective. - // Please see comments of SessionVars.OptObjective for details. - OptObjectiveModerate string = "moderate" - // OptObjectiveDeterminate is a possible value for TiDBOptObjective. - OptObjectiveDeterminate = "determinate" -) - -// ForcePreAggStr means 1st hashagg will be pre aggregated. -// AutoStr means TiFlash will decide which policy for 1st hashagg. -// ForceStreamingStr means 1st hashagg will for pass through all blocks. -const ( - ForcePreAggStr = "force_preagg" - AutoStr = "auto" - ForceStreamingStr = "force_streaming" -) - -const ( - // AllReplicaStr is the string value of AllReplicas. - AllReplicaStr = "all_replicas" - // ClosestAdaptiveStr is the string value of ClosestAdaptive. - ClosestAdaptiveStr = "closest_adaptive" - // ClosestReplicasStr is the string value of ClosestReplicas. - ClosestReplicasStr = "closest_replicas" -) - -const ( - // DispatchPolicyRRStr is string value for DispatchPolicyRR. - DispatchPolicyRRStr = "round_robin" - // DispatchPolicyConsistentHashStr is string value for DispatchPolicyConsistentHash. - DispatchPolicyConsistentHashStr = "consistent_hash" - // DispatchPolicyInvalidStr is string value for DispatchPolicyInvalid. - DispatchPolicyInvalidStr = "invalid" -) - -// ConcurrencyUnset means the value the of the concurrency related variable is unset. -const ConcurrencyUnset = -1 - -// ExchangeCompressionMode means the compress method used in exchange operator -type ExchangeCompressionMode int - -const ( - // ExchangeCompressionModeNONE indicates no compression - ExchangeCompressionModeNONE ExchangeCompressionMode = iota - // ExchangeCompressionModeFast indicates fast compression/decompression speed, compression ratio is lower than HC mode - ExchangeCompressionModeFast - // ExchangeCompressionModeHC indicates high compression (HC) ratio mode - ExchangeCompressionModeHC - // ExchangeCompressionModeUnspecified indicates unspecified compress method, let TiDB choose one - ExchangeCompressionModeUnspecified - - // RecommendedExchangeCompressionMode indicates recommended compression mode - RecommendedExchangeCompressionMode ExchangeCompressionMode = ExchangeCompressionModeFast - - exchangeCompressionModeUnspecifiedName string = "UNSPECIFIED" -) - -// Name returns the name of ExchangeCompressionMode -func (t ExchangeCompressionMode) Name() string { - if t == ExchangeCompressionModeUnspecified { - return exchangeCompressionModeUnspecifiedName - } - return t.ToTipbCompressionMode().String() -} - -// ToExchangeCompressionMode returns the ExchangeCompressionMode from name -func ToExchangeCompressionMode(name string) (ExchangeCompressionMode, bool) { - name = strings.ToUpper(name) - if name == exchangeCompressionModeUnspecifiedName { - return ExchangeCompressionModeUnspecified, true - } - value, ok := tipb.CompressionMode_value[name] - if ok { - return ExchangeCompressionMode(value), true - } - return ExchangeCompressionModeNONE, false -} - -// ToTipbCompressionMode returns tipb.CompressionMode from kv.ExchangeCompressionMode -func (t ExchangeCompressionMode) ToTipbCompressionMode() tipb.CompressionMode { - switch t { - case ExchangeCompressionModeNONE: - return tipb.CompressionMode_NONE - case ExchangeCompressionModeFast: - return tipb.CompressionMode_FAST - case ExchangeCompressionModeHC: - return tipb.CompressionMode_HIGH_COMPRESSION - } - return tipb.CompressionMode_NONE -} - -// ScopeFlag is for system variable whether can be changed in global/session dynamically or not. -type ScopeFlag uint8 - -// TypeFlag is the SysVar type, which doesn't exactly match MySQL types. -type TypeFlag byte - -const ( - // ScopeNone means the system variable can not be changed dynamically. - ScopeNone ScopeFlag = 0 - // ScopeGlobal means the system variable can be changed globally. - ScopeGlobal ScopeFlag = 1 << 0 - // ScopeSession means the system variable can only be changed in current session. - ScopeSession ScopeFlag = 1 << 1 - // ScopeInstance means it is similar to global but doesn't propagate to other TiDB servers. - ScopeInstance ScopeFlag = 1 << 2 - - // TypeStr is the default - TypeStr TypeFlag = iota - // TypeBool for boolean - TypeBool - // TypeInt for integer - TypeInt - // TypeEnum for Enum - TypeEnum - // TypeFloat for Double - TypeFloat - // TypeUnsigned for Unsigned integer - TypeUnsigned - // TypeTime for time of day (a TiDB extension) - TypeTime - // TypeDuration for a golang duration (a TiDB extension) - TypeDuration - - // On is the canonical string for ON - On = "ON" - // Off is the canonical string for OFF - Off = "OFF" - // Warn means return warnings - Warn = "WARN" - // IntOnly means enable for int type - IntOnly = "INT_ONLY" - // Marker is a special log redact behavior - Marker = "MARKER" - - // AssertionStrictStr is a choice of variable TiDBTxnAssertionLevel that means full assertions should be performed, - // even if the performance might be slowed down. - AssertionStrictStr = "STRICT" - // AssertionFastStr is a choice of variable TiDBTxnAssertionLevel that means assertions that doesn't affect - // performance should be performed. - AssertionFastStr = "FAST" - // AssertionOffStr is a choice of variable TiDBTxnAssertionLevel that means no assertion should be performed. - AssertionOffStr = "OFF" - // OOMActionCancel constants represents the valid action configurations for OOMAction "CANCEL". - OOMActionCancel = "CANCEL" - // OOMActionLog constants represents the valid action configurations for OOMAction "LOG". - OOMActionLog = "LOG" - - // TSOClientRPCModeDefault is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client sends batched - // TSO requests serially. - TSOClientRPCModeDefault = "DEFAULT" - // TSOClientRPCModeParallel is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to - // keep approximately 2 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time - // by half, at the expense of about twice the amount of TSO RPC calls. - TSOClientRPCModeParallel = "PARALLEL" - // TSOClientRPCModeParallelFast is a choice of variable TiDBTSOClientRPCMode. In this mode, the TSO client tries to - // keep approximately 4 batched TSO requests running in parallel. This option tries to reduce the batch-waiting time - // by 3/4, at the expense of about 4 times the amount of TSO RPC calls. - TSOClientRPCModeParallelFast = "PARALLEL-FAST" -) - -// Global config name list. -const ( - GlobalConfigEnableTopSQL = "enable_resource_metering" - GlobalConfigSourceID = "source_id" -) - -func (s ScopeFlag) String() string { - var scopes []string - if s == ScopeNone { - return "NONE" - } - if s&ScopeSession != 0 { - scopes = append(scopes, "SESSION") - } - if s&ScopeGlobal != 0 { - scopes = append(scopes, "GLOBAL") - } - if s&ScopeInstance != 0 { - scopes = append(scopes, "INSTANCE") - } - return strings.Join(scopes, ",") -} - -// ClusteredIndexDefMode controls the default clustered property for primary key. -type ClusteredIndexDefMode int - -const ( - // ClusteredIndexDefModeIntOnly indicates only single int primary key will default be clustered. - ClusteredIndexDefModeIntOnly ClusteredIndexDefMode = 0 - // ClusteredIndexDefModeOn indicates primary key will default be clustered. - ClusteredIndexDefModeOn ClusteredIndexDefMode = 1 - // ClusteredIndexDefModeOff indicates primary key will default be non-clustered. - ClusteredIndexDefModeOff ClusteredIndexDefMode = 2 -) - -// TiDBOptEnableClustered converts enable clustered options to ClusteredIndexDefMode. -func TiDBOptEnableClustered(opt string) ClusteredIndexDefMode { - switch opt { - case On: - return ClusteredIndexDefModeOn - case Off: - return ClusteredIndexDefModeOff - default: - return ClusteredIndexDefModeIntOnly - } -} - -const ( - // ScatterOff means default, will not scatter region - ScatterOff string = "" - // ScatterTable means scatter region at table level - ScatterTable string = "table" - // ScatterGlobal means scatter region at global level - ScatterGlobal string = "global" -) - -const ( - // PlacementModeStrict indicates all placement operations should be checked strictly in ddl - PlacementModeStrict string = "STRICT" - // PlacementModeIgnore indicates ignore all placement operations in ddl - PlacementModeIgnore string = "IGNORE" -) - -const ( - // LocalDayTimeFormat is the local format of analyze start time and end time. - LocalDayTimeFormat = "15:04" - // FullDayTimeFormat is the full format of analyze start time and end time. - FullDayTimeFormat = "15:04 -0700" -) - -// SetDDLReorgWorkerCounter sets DDLReorgWorkerCounter count. -// Sysvar validation enforces the range to already be correct. -func SetDDLReorgWorkerCounter(cnt int32) { - goatomic.StoreInt32(&DDLReorgWorkerCounter, cnt) -} - -// GetDDLReorgWorkerCounter gets DDLReorgWorkerCounter. -func GetDDLReorgWorkerCounter() int32 { - return goatomic.LoadInt32(&DDLReorgWorkerCounter) -} - -// SetDDLFlashbackConcurrency sets DDLFlashbackConcurrency count. -// Sysvar validation enforces the range to already be correct. -func SetDDLFlashbackConcurrency(cnt int32) { - goatomic.StoreInt32(&DDLFlashbackConcurrency, cnt) -} - -// GetDDLFlashbackConcurrency gets DDLFlashbackConcurrency count. -func GetDDLFlashbackConcurrency() int32 { - return goatomic.LoadInt32(&DDLFlashbackConcurrency) -} - -// SetDDLReorgBatchSize sets DDLReorgBatchSize size. -// Sysvar validation enforces the range to already be correct. -func SetDDLReorgBatchSize(cnt int32) { - goatomic.StoreInt32(&DDLReorgBatchSize, cnt) -} - -// GetDDLReorgBatchSize gets DDLReorgBatchSize. -func GetDDLReorgBatchSize() int32 { - return goatomic.LoadInt32(&DDLReorgBatchSize) -} - -// SetDDLErrorCountLimit sets ddlErrorCountlimit size. -func SetDDLErrorCountLimit(cnt int64) { - goatomic.StoreInt64(&DDLErrorCountLimit, cnt) -} - -// GetDDLErrorCountLimit gets ddlErrorCountlimit size. -func GetDDLErrorCountLimit() int64 { - return goatomic.LoadInt64(&DDLErrorCountLimit) -} - -// SetDDLReorgRowFormat sets DDLReorgRowFormat version. -func SetDDLReorgRowFormat(format int64) { - goatomic.StoreInt64(&DDLReorgRowFormat, format) -} - -// GetDDLReorgRowFormat gets DDLReorgRowFormat version. -func GetDDLReorgRowFormat() int64 { - return goatomic.LoadInt64(&DDLReorgRowFormat) -} - -// SetMaxDeltaSchemaCount sets MaxDeltaSchemaCount size. -func SetMaxDeltaSchemaCount(cnt int64) { - goatomic.StoreInt64(&MaxDeltaSchemaCount, cnt) -} - -// GetMaxDeltaSchemaCount gets MaxDeltaSchemaCount size. -func GetMaxDeltaSchemaCount() int64 { - return goatomic.LoadInt64(&MaxDeltaSchemaCount) -} diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index 3c8006acea7e0..394cc9ea8ea49 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -1257,6 +1257,9 @@ const ( // TiDBCircuitBreakerPDMetadataErrorRateThresholdRatio variable is used to set ratio of errors to trip the circuit breaker for get region calls to PD // https://github.com/tikv/rfcs/blob/master/text/0115-circuit-breaker.md TiDBCircuitBreakerPDMetadataErrorRateThresholdRatio = "tidb_cb_pd_metadata_error_rate_threshold_ratio" + + // TiDBAccelerateUserCreationUpdate decides whether tidb will load & update the whole user's data in-memory. + TiDBAccelerateUserCreationUpdate = "tidb_accelerate_user_creation_update" ) // TiDB intentional limits @@ -1631,6 +1634,7 @@ const ( DefTiDBAdvancerCheckPointLagLimit = 48 * time.Hour DefTiDBIndexLookUpPushDownPolicy = IndexLookUpPushDownPolicyHintOnly DefTiDBCircuitBreakerPDMetaErrorRateRatio = 0.0 + DefTiDBAccelerateUserCreationUpdate = false ) // Process global variables. @@ -1756,6 +1760,7 @@ var ( AdvancerCheckPointLagLimit = atomic.NewDuration(DefTiDBAdvancerCheckPointLagLimit) CircuitBreakerPDMetadataErrorRateThresholdRatio = atomic.NewFloat64(0.0) + AccelerateUserCreationUpdate = atomic.NewBool(DefTiDBAccelerateUserCreationUpdate) ) var ( From a80aa74d915196ca1ddc2571e235e0a68fbaaae5 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 28 Jan 2026 20:33:30 +0800 Subject: [PATCH 07/17] make fmt & make bazel_prepare --- pkg/privilege/privileges/cache.go | 19 ++++++++++++------- pkg/privilege/privileges/cache_test.go | 16 +++++++++------- pkg/privilege/privileges/privileges_test.go | 4 ++-- .../privileges/tidb_auth_token_test.go | 3 ++- pkg/session/bootstrap.go | 2 +- pkg/sessionctx/variable/tidb_vars.go | 2 +- 6 files changed, 27 insertions(+), 19 deletions(-) diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 6b669a4d729a2..d04c201c22ab4 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -257,7 +257,7 @@ func (g roleGraphEdgesTable) Find(user, host string) bool { if host == "" { host = "%" } - key := auth.RoleIdentity{user, host} + key := auth.RoleIdentity{user, host} if g.roleList == nil { return false } @@ -412,7 +412,7 @@ func (p *MySQLPrivilege) FindAllRole(activeRoles []*auth.RoleIdentity) []*auth.R if _, ok := visited[role.String()]; !ok { visited[role.String()] = true ret = append(ret, role) - key := auth.RoleIdentity{role.Username, role.Hostname} + key := auth.RoleIdentity{role.Username, role.Hostname} if edgeTable, ok := p.roleGraph[key]; ok { for _, v := range edgeTable.roleList { if _, ok := visited[v.String()]; !ok { @@ -431,7 +431,7 @@ func (p *MySQLPrivilege) FindRole(user string, host string, role *auth.RoleIdent rec := p.matchUser(user, host) r := p.matchUser(role.Username, role.Hostname) if rec != nil && r != nil { - key := auth.RoleIdentity{rec.User, rec.Host} + key := auth.RoleIdentity{rec.User, rec.Host} return p.roleGraph[key].Find(role.Username, role.Hostname) } return false @@ -1247,8 +1247,8 @@ func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.Resul toUser = row.GetString(i) } } - fromKey := auth.RoleIdentity{fromUser, fromHost} - toKey := auth.RoleIdentity{toUser, toHost} + fromKey := auth.RoleIdentity{fromUser, fromHost} + toKey := auth.RoleIdentity{toUser, toHost} roleGraph, ok := p.roleGraph[toKey] if !ok { roleGraph = roleGraphEdgesTable{roleList: make(map[auth.RoleIdentity]*auth.RoleIdentity)} @@ -1830,7 +1830,7 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r slices.Sort(gs[sortFromIdx:]) // Show role grants. - graphKey := auth.RoleIdentity{user, host} + graphKey := auth.RoleIdentity{user, host} edgeTable, ok := p.roleGraph[graphKey] g = "" if ok { @@ -2063,7 +2063,7 @@ func (p *MySQLPrivilege) getDefaultRoles(user, host string) []*auth.RoleIdentity } func (p *MySQLPrivilege) getAllRoles(user, host string) []*auth.RoleIdentity { - key := auth.RoleIdentity{user, host} + key := auth.RoleIdentity{user, host} edgeTable, ok := p.roleGraph[key] ret := make([]*auth.RoleIdentity, 0, len(edgeTable.roleList)) if ok { @@ -2074,6 +2074,11 @@ func (p *MySQLPrivilege) getAllRoles(user, host string) []*auth.RoleIdentity { return ret } +// SetGlobalVarsAccessor is only used for test. +func (p *MySQLPrivilege) SetGlobalVarsAccessor(globalVars variable.GlobalVarAccessor) { + p.globalVars = globalVars +} + // Handle wraps MySQLPrivilege providing thread safe access. type Handle struct { sctx util.SessionPool diff --git a/pkg/privilege/privileges/cache_test.go b/pkg/privilege/privileges/cache_test.go index a2181b59807c4..1c30d3d744810 100644 --- a/pkg/privilege/privileges/cache_test.go +++ b/pkg/privilege/privileges/cache_test.go @@ -15,6 +15,7 @@ package privileges_test import ( + "context" "fmt" "testing" "time" @@ -22,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/privilege/privileges" + "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" @@ -71,7 +73,7 @@ func TestLoadUserTable(t *testing.T) { for _, plugin := range []string{mysql.AuthNativePassword, mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password} { p = privileges.NewMySQLPrivilege() p.SetGlobalVarsAccessor(se.GetSessionVars().GlobalVarsAccessor) - require.NoError(t, se.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), vardef.DefaultAuthPlugin, plugin)) + require.NoError(t, se.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.DefaultAuthPlugin, plugin)) require.NoError(t, p.LoadUserTable(se.GetSQLExecutor())) require.Equal(t, plugin, p.User()[0].AuthPlugin) } @@ -313,13 +315,13 @@ func TestLoadRoleGraph(t *testing.T) { p = privileges.NewMySQLPrivilege() require.NoError(t, p.LoadRoleGraph(se.GetSQLExecutor())) graph := p.RoleGraph() - require.True(t, graph["root@%"].Find("r_2", "%")) - require.True(t, graph["root@%"].Find("r_4", "%")) - require.True(t, graph["user2@%"].Find("r_1", "%")) - require.True(t, graph["user1@%"].Find("r_3", "%")) - _, ok := graph["illedal"] + require.True(t, graph[auth.RoleIdentity{"root", "%"}].Find("r_2", "%")) + require.True(t, graph[auth.RoleIdentity{"root", "%"}].Find("r_4", "%")) + require.True(t, graph[auth.RoleIdentity{"user2", "%"}].Find("r_1", "%")) + require.True(t, graph[auth.RoleIdentity{"user1", "%"}].Find("r_3", "%")) + _, ok := graph[auth.RoleIdentity{Username: "illedal"}] require.False(t, ok) - require.False(t, graph["root@%"].Find("r_1", "%")) + require.False(t, graph[auth.RoleIdentity{"root", "%"}].Find("r_1", "%")) } func TestRoleGraphBFS(t *testing.T) { diff --git a/pkg/privilege/privileges/privileges_test.go b/pkg/privilege/privileges/privileges_test.go index 9fd80b0176d16..9c72663db4496 100644 --- a/pkg/privilege/privileges/privileges_test.go +++ b/pkg/privilege/privileges/privileges_test.go @@ -1905,7 +1905,7 @@ func TestCheckPasswordExpired(t *testing.T) { sessionVars := variable.NewSessionVars(nil) sessionVars.GlobalVarsAccessor = variable.NewMockGlobalAccessor4Tests() record := privileges.NewUserRecord("%", "root") - userPrivilege := privileges.NewUserPrivileges(privileges.NewHandle(nil), nil) + userPrivilege := privileges.NewUserPrivileges(privileges.NewHandle(nil, nil), nil) record.PasswordExpired = true _, err := userPrivilege.CheckPasswordExpired(sessionVars, &record) @@ -2168,7 +2168,7 @@ func TestSQLVariableAccelerateUserCreationUpdate(t *testing.T) { // 2. change the variable and check tk.MustExec("set @@global.tidb_accelerate_user_creation_update = on") tk.MustQuery("select @@global.tidb_accelerate_user_creation_update").Check(testkit.Rows("1")) - require.True(t, vardef.AccelerateUserCreationUpdate.Load()) + require.True(t, variable.AccelerateUserCreationUpdate.Load()) tk.MustExec("create user bbb") handle.CheckFullData(t, false) // trigger priv reload, but data for bbb is not really loaded diff --git a/pkg/privilege/privileges/tidb_auth_token_test.go b/pkg/privilege/privileges/tidb_auth_token_test.go index a16d5c38fb959..1612d52774015 100644 --- a/pkg/privilege/privileges/tidb_auth_token_test.go +++ b/pkg/privilege/privileges/tidb_auth_token_test.go @@ -30,6 +30,7 @@ import ( jwsRepo "github.com/lestrrat-go/jwx/v2/jws" jwtRepo "github.com/lestrrat-go/jwx/v2/jwt" "github.com/lestrrat-go/jwx/v2/jwt/openid" + "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/util/hack" "github.com/stretchr/testify/require" ) @@ -482,7 +483,7 @@ func (p *MySQLPrivilege) GlobalPriv(user string) []globalPrivRecord { return ret.data } -func (p *MySQLPrivilege) RoleGraph() map[string]roleGraphEdgesTable { +func (p *MySQLPrivilege) RoleGraph() map[auth.RoleIdentity]roleGraphEdgesTable { return p.roleGraph } diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index a3ad2c9ce872b..f26f2c211c701 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -1269,7 +1269,7 @@ func DisableRunBootstrapSQLFileInTest() { } var ( - bootstrapVersion = []func(sessiontypes.Session, int64) { + bootstrapVersion = []func(sessiontypes.Session, int64){ upgradeToVer2, upgradeToVer3, upgradeToVer4, diff --git a/pkg/sessionctx/variable/tidb_vars.go b/pkg/sessionctx/variable/tidb_vars.go index 394cc9ea8ea49..a9f7669ef77bf 100644 --- a/pkg/sessionctx/variable/tidb_vars.go +++ b/pkg/sessionctx/variable/tidb_vars.go @@ -1760,7 +1760,7 @@ var ( AdvancerCheckPointLagLimit = atomic.NewDuration(DefTiDBAdvancerCheckPointLagLimit) CircuitBreakerPDMetadataErrorRateThresholdRatio = atomic.NewFloat64(0.0) - AccelerateUserCreationUpdate = atomic.NewBool(DefTiDBAccelerateUserCreationUpdate) + AccelerateUserCreationUpdate = atomic.NewBool(DefTiDBAccelerateUserCreationUpdate) ) var ( From 93fcf1576ef4e2f1e3ce1f2912b2cb478042d0c3 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 28 Jan 2026 21:55:45 +0800 Subject: [PATCH 08/17] make lint happy --- pkg/privilege/privileges/cache.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index d04c201c22ab4..3d5e8420eb0e4 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -257,7 +257,7 @@ func (g roleGraphEdgesTable) Find(user, host string) bool { if host == "" { host = "%" } - key := auth.RoleIdentity{user, host} + key := auth.RoleIdentity{Username: user, Hostname: host} if g.roleList == nil { return false } @@ -412,7 +412,7 @@ func (p *MySQLPrivilege) FindAllRole(activeRoles []*auth.RoleIdentity) []*auth.R if _, ok := visited[role.String()]; !ok { visited[role.String()] = true ret = append(ret, role) - key := auth.RoleIdentity{role.Username, role.Hostname} + key := auth.RoleIdentity{Username: role.Username, Hostname: role.Hostname} if edgeTable, ok := p.roleGraph[key]; ok { for _, v := range edgeTable.roleList { if _, ok := visited[v.String()]; !ok { @@ -431,7 +431,7 @@ func (p *MySQLPrivilege) FindRole(user string, host string, role *auth.RoleIdent rec := p.matchUser(user, host) r := p.matchUser(role.Username, role.Hostname) if rec != nil && r != nil { - key := auth.RoleIdentity{rec.User, rec.Host} + key := auth.RoleIdentity{Username: rec.User, Hostname: rec.Host} return p.roleGraph[key].Find(role.Username, role.Hostname) } return false @@ -1247,8 +1247,8 @@ func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.Resul toUser = row.GetString(i) } } - fromKey := auth.RoleIdentity{fromUser, fromHost} - toKey := auth.RoleIdentity{toUser, toHost} + fromKey := auth.RoleIdentity{Username: fromUser, Hostname: fromHost} + toKey := auth.RoleIdentity{Username: toUser, Hostname: toHost} roleGraph, ok := p.roleGraph[toKey] if !ok { roleGraph = roleGraphEdgesTable{roleList: make(map[auth.RoleIdentity]*auth.RoleIdentity)} @@ -1830,7 +1830,7 @@ func (p *MySQLPrivilege) showGrants(ctx sessionctx.Context, user, host string, r slices.Sort(gs[sortFromIdx:]) // Show role grants. - graphKey := auth.RoleIdentity{user, host} + graphKey := auth.RoleIdentity{Username: user, Hostname: host} edgeTable, ok := p.roleGraph[graphKey] g = "" if ok { @@ -2063,7 +2063,7 @@ func (p *MySQLPrivilege) getDefaultRoles(user, host string) []*auth.RoleIdentity } func (p *MySQLPrivilege) getAllRoles(user, host string) []*auth.RoleIdentity { - key := auth.RoleIdentity{user, host} + key := auth.RoleIdentity{Username: user, Hostname: host} edgeTable, ok := p.roleGraph[key] ret := make([]*auth.RoleIdentity, 0, len(edgeTable.roleList)) if ok { From 7878194cd17fd82bcbf612ef3e774643f9cdc6fd Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 10:08:12 +0800 Subject: [PATCH 09/17] make lint happy --- pkg/privilege/privileges/cache_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/privilege/privileges/cache_test.go b/pkg/privilege/privileges/cache_test.go index 1c30d3d744810..99613084d3533 100644 --- a/pkg/privilege/privileges/cache_test.go +++ b/pkg/privilege/privileges/cache_test.go @@ -315,13 +315,13 @@ func TestLoadRoleGraph(t *testing.T) { p = privileges.NewMySQLPrivilege() require.NoError(t, p.LoadRoleGraph(se.GetSQLExecutor())) graph := p.RoleGraph() - require.True(t, graph[auth.RoleIdentity{"root", "%"}].Find("r_2", "%")) - require.True(t, graph[auth.RoleIdentity{"root", "%"}].Find("r_4", "%")) - require.True(t, graph[auth.RoleIdentity{"user2", "%"}].Find("r_1", "%")) - require.True(t, graph[auth.RoleIdentity{"user1", "%"}].Find("r_3", "%")) + require.True(t, graph[auth.RoleIdentity{Username: "root", Hostname: "%"}].Find("r_2", "%")) + require.True(t, graph[auth.RoleIdentity{Username: "root", Hostname: "%"}].Find("r_4", "%")) + require.True(t, graph[auth.RoleIdentity{Username: "user2", Hostname: "%"}].Find("r_1", "%")) + require.True(t, graph[auth.RoleIdentity{Username: "user1", Hostname: "%"}].Find("r_3", "%")) _, ok := graph[auth.RoleIdentity{Username: "illedal"}] require.False(t, ok) - require.False(t, graph[auth.RoleIdentity{"root", "%"}].Find("r_1", "%")) + require.False(t, graph[auth.RoleIdentity{Username: "root", Hostname: "%"}].Find("r_1", "%")) } func TestRoleGraphBFS(t *testing.T) { From f42e57d52a93830b4a42af5a20b95b48f6f44f90 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 12:11:05 +0800 Subject: [PATCH 10/17] fix test case TestEnsureActiveUserCoverage --- pkg/executor/show.go | 8 ++-- pkg/executor/simple.go | 42 ++++++++++----------- pkg/privilege/privilege.go | 7 ++-- pkg/privilege/privileges/cache.go | 13 +++++-- pkg/privilege/privileges/privileges.go | 30 +++++++-------- pkg/privilege/privileges/privileges_test.go | 2 +- 6 files changed, 54 insertions(+), 48 deletions(-) diff --git a/pkg/executor/show.go b/pkg/executor/show.go index 0cb952c5ae713..f0f56ea113675 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -207,7 +207,7 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { case ast.ShowEngines: return e.fetchShowEngines(ctx) case ast.ShowGrants: - return e.fetchShowGrants() + return e.fetchShowGrants(ctx) case ast.ShowIndex: return e.fetchShowIndex() case ast.ShowProcedureStatus: @@ -1869,7 +1869,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error { return nil } -func (e *ShowExec) fetchShowGrants() error { +func (e *ShowExec) fetchShowGrants(ctx context.Context) error { vars := e.Ctx().GetSessionVars() checker := privilege.GetPrivilegeManager(e.Ctx()) if checker == nil { @@ -1898,11 +1898,11 @@ func (e *ShowExec) fetchShowGrants() error { if r.Hostname == "" { r.Hostname = "%" } - if !checker.FindEdge(e.Ctx(), r, e.User) { + if !checker.FindEdge(ctx, e.Ctx(), r, e.User) { return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(r.String(), e.User.String()) } } - gs, err := checker.ShowGrants(e.Ctx(), e.User, e.Roles) + gs, err := checker.ShowGrants(ctx, e.Ctx(), e.User, e.Roles) if err != nil { return errors.Trace(err) } diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 403edfb7f28d9..c221d5860bc0f 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -176,7 +176,7 @@ func (e *SimpleExec) Next(ctx context.Context, _ *chunk.Chunk) (err error) { case *ast.DropStatsStmt: err = e.executeDropStats(ctx, x) case *ast.SetRoleStmt: - err = e.executeSetRole(x) + err = e.executeSetRole(ctx, x) case *ast.RevokeRoleStmt: err = e.executeRevokeRole(ctx, x) case *ast.SetDefaultRoleStmt: @@ -274,7 +274,7 @@ func (e *SimpleExec) setDefaultRoleRegular(ctx context.Context, s *ast.SetDefaul } for _, role := range s.RoleList { checker := privilege.GetPrivilegeManager(e.Ctx()) - ok := checker.FindEdge(e.Ctx(), role, user) + ok := checker.FindEdge(ctx, e.Ctx(), role, user) if !ok { if _, rollbackErr := sqlExecutor.ExecuteInternal(internalCtx, "rollback"); rollbackErr != nil { return rollbackErr @@ -388,7 +388,7 @@ func (e *SimpleExec) setDefaultRoleForCurrentUser(s *ast.SetDefaultRoleStmt) (er if i > 0 { sqlescape.MustFormatSQL(sql, ",") } - ok := checker.FindEdge(e.Ctx(), role, user) + ok := checker.FindEdge(ctx, e.Ctx(), role, user) if !ok { return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(role.String(), user.String()) } @@ -458,7 +458,7 @@ func (e *SimpleExec) executeSetDefaultRole(ctx context.Context, s *ast.SetDefaul return domain.GetDomain(e.Ctx()).NotifyUpdatePrivilege(users) } -func (e *SimpleExec) setRoleRegular(s *ast.SetRoleStmt) error { +func (e *SimpleExec) setRoleRegular(ctx context.Context, s *ast.SetRoleStmt) error { // Deal with SQL like `SET ROLE role1, role2;` checkDup := make(map[string]*auth.RoleIdentity, len(s.RoleList)) // Check whether RoleNameList contain duplicate role name. @@ -472,7 +472,7 @@ func (e *SimpleExec) setRoleRegular(s *ast.SetRoleStmt) error { } checker := privilege.GetPrivilegeManager(e.Ctx()) - ok, roleName := checker.ActiveRoles(e.Ctx(), roleList) + ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), roleList) if !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) @@ -480,12 +480,12 @@ func (e *SimpleExec) setRoleRegular(s *ast.SetRoleStmt) error { return nil } -func (e *SimpleExec) setRoleAll() error { +func (e *SimpleExec) setRoleAll(ctx context.Context) error { // Deal with SQL like `SET ROLE ALL;` checker := privilege.GetPrivilegeManager(e.Ctx()) user, host := e.Ctx().GetSessionVars().User.AuthUsername, e.Ctx().GetSessionVars().User.AuthHostname roles := checker.GetAllRoles(user, host) - ok, roleName := checker.ActiveRoles(e.Ctx(), roles) + ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), roles) if !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) @@ -493,7 +493,7 @@ func (e *SimpleExec) setRoleAll() error { return nil } -func (e *SimpleExec) setRoleAllExcept(s *ast.SetRoleStmt) error { +func (e *SimpleExec) setRoleAllExcept(ctx context.Context, s *ast.SetRoleStmt) error { // Deal with SQL like `SET ROLE ALL EXCEPT role1, role2;` for _, r := range s.RoleList { if r.Hostname == "" { @@ -524,7 +524,7 @@ func (e *SimpleExec) setRoleAllExcept(s *ast.SetRoleStmt) error { } afterExcept := filter(roles, banned) - ok, roleName := checker.ActiveRoles(e.Ctx(), afterExcept) + ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), afterExcept) if !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) @@ -532,12 +532,12 @@ func (e *SimpleExec) setRoleAllExcept(s *ast.SetRoleStmt) error { return nil } -func (e *SimpleExec) setRoleDefault() error { +func (e *SimpleExec) setRoleDefault(ctx context.Context) error { // Deal with SQL like `SET ROLE DEFAULT;` checker := privilege.GetPrivilegeManager(e.Ctx()) user, host := e.Ctx().GetSessionVars().User.AuthUsername, e.Ctx().GetSessionVars().User.AuthHostname roles := checker.GetDefaultRoles(user, host) - ok, roleName := checker.ActiveRoles(e.Ctx(), roles) + ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), roles) if !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) @@ -545,11 +545,11 @@ func (e *SimpleExec) setRoleDefault() error { return nil } -func (e *SimpleExec) setRoleNone() error { +func (e *SimpleExec) setRoleNone(ctx context.Context) error { // Deal with SQL like `SET ROLE NONE;` checker := privilege.GetPrivilegeManager(e.Ctx()) roles := make([]*auth.RoleIdentity, 0) - ok, roleName := checker.ActiveRoles(e.Ctx(), roles) + ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), roles) if !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) @@ -557,18 +557,18 @@ func (e *SimpleExec) setRoleNone() error { return nil } -func (e *SimpleExec) executeSetRole(s *ast.SetRoleStmt) error { +func (e *SimpleExec) executeSetRole(ctx context.Context, s *ast.SetRoleStmt) error { switch s.SetRoleOpt { case ast.SetRoleRegular: - return e.setRoleRegular(s) + return e.setRoleRegular(ctx, s) case ast.SetRoleAll: - return e.setRoleAll() + return e.setRoleAll(ctx) case ast.SetRoleAllExcept: - return e.setRoleAllExcept(s) + return e.setRoleAllExcept(ctx, s) case ast.SetRoleNone: - return e.setRoleNone() + return e.setRoleNone(ctx) case ast.SetRoleDefault: - return e.setRoleDefault() + return e.setRoleDefault(ctx) } return nil } @@ -764,7 +764,7 @@ func (e *SimpleExec) executeRevokeRole(ctx context.Context, s *ast.RevokeRoleStm if checker == nil { return errors.New("miss privilege checker") } - if ok, roleName := checker.ActiveRoles(e.Ctx(), activeRoles); !ok { + if ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), activeRoles); !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) } @@ -2428,7 +2428,7 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e } if s.IsDropRole { // apply new activeRoles - if ok, roleName := checker.ActiveRoles(e.Ctx(), activeRoles); !ok { + if ok, roleName := checker.ActiveRoles(ctx, e.Ctx(), activeRoles); !ok { u := e.Ctx().GetSessionVars().User return exeerrors.ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) } diff --git a/pkg/privilege/privilege.go b/pkg/privilege/privilege.go index 1591700e2238d..5452f3ee5a5a1 100644 --- a/pkg/privilege/privilege.go +++ b/pkg/privilege/privilege.go @@ -15,6 +15,7 @@ package privilege import ( + "context" "fmt" "github.com/pingcap/tidb/pkg/parser/auth" @@ -45,7 +46,7 @@ type VerificationInfo struct { // Manager is the interface for providing privilege related operations. type Manager interface { // ShowGrants shows granted privileges for user. - ShowGrants(ctx sessionctx.Context, user *auth.UserIdentity, roles []*auth.RoleIdentity) ([]string, error) + ShowGrants(ctx context.Context, sctx sessionctx.Context, user *auth.UserIdentity, roles []*auth.RoleIdentity) ([]string, error) // GetEncodedPassword shows the encoded password for user. GetEncodedPassword(user, host string) string @@ -102,10 +103,10 @@ type Manager interface { // ActiveRoles active roles for current session. // The first illegal role will be returned. - ActiveRoles(ctx sessionctx.Context, roleList []*auth.RoleIdentity) (bool, string) + ActiveRoles(ctx context.Context, sctx sessionctx.Context, roleList []*auth.RoleIdentity) (bool, string) // FindEdge find if there is an edge between role and user. - FindEdge(ctx sessionctx.Context, role *auth.RoleIdentity, user *auth.UserIdentity) bool + FindEdge(ctx context.Context, sctx sessionctx.Context, role *auth.RoleIdentity, user *auth.UserIdentity) bool // GetDefaultRoles returns all default roles for certain user. GetDefaultRoles(user, host string) []*auth.RoleIdentity diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 3d5e8420eb0e4..9ab17e1dc54d6 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -437,9 +437,9 @@ func (p *MySQLPrivilege) FindRole(user string, host string, role *auth.RoleIdent return false } -func findRole(h *Handle, user string, host string, role *auth.RoleIdentity) bool { - terror.Log(h.ensureActiveUser(user)) - terror.Log(h.ensureActiveUser(role.Username)) +func findRole(ctx context.Context, h *Handle, user string, host string, role *auth.RoleIdentity) bool { + terror.Log(h.ensureActiveUser(ctx, user)) + terror.Log(h.ensureActiveUser(ctx, role.Username)) mysqlPrivilege := h.Get() return mysqlPrivilege.FindRole(user, host, role) } @@ -2101,7 +2101,12 @@ func NewHandle(sctx util.SessionPool, globalVars variable.GlobalVarAccessor) *Ha } // ensureActiveUser ensure that the specific user data is loaded in-memory. -func (h *Handle) ensureActiveUser(user string) error { +func (h *Handle) ensureActiveUser(ctx context.Context, user string) error { + if p := ctx.Value("mock"); p != nil { + visited := p.(*bool) + *visited = true + } + if h.fullData.Load() { // All users data are in-memory, nothing to do return nil diff --git a/pkg/privilege/privileges/privileges.go b/pkg/privilege/privileges/privileges.go index 8e30fd4748df6..5d52dbe64142e 100644 --- a/pkg/privilege/privileges/privileges.go +++ b/pkg/privilege/privileges/privileges.go @@ -106,7 +106,7 @@ func (p *UserPrivileges) RequestDynamicVerificationWithUser(privName string, gra return false } - terror.Log(p.Handle.ensureActiveUser(user.Username)) + terror.Log(p.Handle.ensureActiveUser(context.Background(), user.Username)) mysqlPriv := p.Handle.Get() roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) return mysqlPriv.RequestDynamicVerification(roles, user.Username, user.Hostname, privName, grantable) @@ -223,7 +223,7 @@ func (p *UserPrivileges) RequestVerificationWithUser(db, table, column string, p return true } - terror.Log(p.Handle.ensureActiveUser(user.Username)) + terror.Log(p.Handle.ensureActiveUser(context.Background(), user.Username)) mysqlPriv := p.Handle.Get() roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) return mysqlPriv.RequestVerification(roles, user.Username, user.Hostname, db, table, column, priv) @@ -319,7 +319,7 @@ func (p *UserPrivileges) isValidHash(record *UserRecord) bool { // GetEncodedPassword implements the Manager interface. func (p *UserPrivileges) GetEncodedPassword(user, host string) string { - terror.Log(p.Handle.ensureActiveUser(user)) + terror.Log(p.Handle.ensureActiveUser(context.Background(), user)) mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { @@ -339,7 +339,7 @@ func (p *UserPrivileges) GetAuthPluginForConnection(user, host string) (string, return mysql.AuthNativePassword, nil } - terror.Log(p.Handle.ensureActiveUser(user)) + terror.Log(p.Handle.ensureActiveUser(context.Background(), user)) mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { @@ -371,7 +371,7 @@ func (p *UserPrivileges) GetAuthPlugin(user, host string) (string, error) { return mysql.AuthNativePassword, nil } - terror.Log(p.Handle.ensureActiveUser(user)) + terror.Log(p.Handle.ensureActiveUser(context.Background(), user)) mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { @@ -388,7 +388,7 @@ func (p *UserPrivileges) MatchIdentity(user, host string, skipNameResolve bool) if SkipWithGrant { return user, host, true } - if err := p.Handle.ensureActiveUser(user); err != nil { + if err := p.Handle.ensureActiveUser(context.Background(), user); err != nil { logutil.BgLogger().Error("ensure user data fail", zap.String("user", user)) } @@ -920,7 +920,7 @@ func (p *UserPrivileges) UserPrivilegesTable(activeRoles []*auth.RoleIdentity, u } // ShowGrants implements privilege.Manager ShowGrants interface. -func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdentity, roles []*auth.RoleIdentity) (grants []string, err error) { +func (p *UserPrivileges) ShowGrants(ctx context.Context, sctx sessionctx.Context, user *auth.UserIdentity, roles []*auth.RoleIdentity) (grants []string, err error) { if SkipWithGrant { return nil, ErrNonexistingGrant.GenWithStackByArgs("root", "%") } @@ -930,11 +930,11 @@ func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdent u = user.AuthUsername h = user.AuthHostname } - if err := p.Handle.ensureActiveUser(u); err != nil { + if err := p.Handle.ensureActiveUser(ctx, u); err != nil { return nil, err } mysqlPrivilege := p.Handle.Get() - grants = mysqlPrivilege.showGrants(ctx, u, h, roles) + grants = mysqlPrivilege.showGrants(sctx, u, h, roles) if len(grants) == 0 { err = ErrNonexistingGrant.GenWithStackByArgs(u, h) } @@ -943,29 +943,29 @@ func (p *UserPrivileges) ShowGrants(ctx sessionctx.Context, user *auth.UserIdent } // ActiveRoles implements privilege.Manager ActiveRoles interface. -func (p *UserPrivileges) ActiveRoles(ctx sessionctx.Context, roleList []*auth.RoleIdentity) (bool, string) { +func (p *UserPrivileges) ActiveRoles(ctx context.Context, sctx sessionctx.Context, roleList []*auth.RoleIdentity) (bool, string) { if SkipWithGrant { return true, "" } u := p.user h := p.host for _, r := range roleList { - ok := findRole(p.Handle, u, h, r) + ok := findRole(ctx, p.Handle, u, h, r) if !ok { logutil.BgLogger().Error("find role failed", zap.Stringer("role", r)) return false, r.String() } } - ctx.GetSessionVars().ActiveRoles = roleList + sctx.GetSessionVars().ActiveRoles = roleList return true, "" } // FindEdge implements privilege.Manager FindRelationship interface. -func (p *UserPrivileges) FindEdge(ctx sessionctx.Context, role *auth.RoleIdentity, user *auth.UserIdentity) bool { +func (p *UserPrivileges) FindEdge(ctx context.Context, sctx sessionctx.Context, role *auth.RoleIdentity, user *auth.UserIdentity) bool { if SkipWithGrant { return false } - ok := findRole(p.Handle, user.Username, user.Hostname, role) + ok := findRole(ctx, p.Handle, user.Username, user.Hostname, role) if !ok { logutil.BgLogger().Error("find role failed", zap.Stringer("role", role)) return false @@ -978,7 +978,7 @@ func (p *UserPrivileges) GetDefaultRoles(user, host string) []*auth.RoleIdentity if SkipWithGrant { return make([]*auth.RoleIdentity, 0, 10) } - terror.Log(p.Handle.ensureActiveUser(user)) + terror.Log(p.Handle.ensureActiveUser(context.Background(), user)) mysqlPrivilege := p.Handle.Get() ret := mysqlPrivilege.getDefaultRoles(user, host) return ret diff --git a/pkg/privilege/privileges/privileges_test.go b/pkg/privilege/privileges/privileges_test.go index 9c72663db4496..8ff62916d57cf 100644 --- a/pkg/privilege/privileges/privileges_test.go +++ b/pkg/privilege/privileges/privileges_test.go @@ -2092,7 +2092,7 @@ func TestNilHandleInConnectionVerification(t *testing.T) { func testShowGrantsSQLMode(t *testing.T, tk *testkit.TestKit, expected []string) { pc := privilege.GetPrivilegeManager(tk.Session()) - gs, err := pc.ShowGrants(tk.Session(), &auth.UserIdentity{Username: "show_sql_mode", Hostname: "localhost"}, nil) + gs, err := pc.ShowGrants(context.Background(), tk.Session(), &auth.UserIdentity{Username: "show_sql_mode", Hostname: "localhost"}, nil) require.NoError(t, err) require.Len(t, gs, 2) require.True(t, testutil.CompareUnorderedStringSlice(gs, expected), fmt.Sprintf("gs: %v, expected: %v", gs, expected)) From baf1a3ae6aacf585684236a972524cc527851760 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 13:53:54 +0800 Subject: [PATCH 11/17] fix test --- pkg/executor/simple.go | 60 +++++++++++++++++++++++++++++++----------- 1 file changed, 45 insertions(+), 15 deletions(-) diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index c221d5860bc0f..562b386e81766 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -1774,15 +1774,25 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) if !(hasCreateUserPriv || hasSystemSchemaPriv) { return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER") } - if checker.RequestDynamicVerificationWithUser("SYSTEM_USER", false, spec.User) && !(hasSystemUserPriv || hasRestrictedUserPriv) { + hasSystemUser, err := userHasDynamicPrivilegeInternal(ctx, sqlExecutor, spec.User, "SYSTEM_USER") + if err != nil { + return err + } + if hasSystemUser && !(hasSystemUserPriv || hasRestrictedUserPriv) { return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("SYSTEM_USER or SUPER") } - if sem.IsEnabled() && checker.RequestDynamicVerificationWithUser("RESTRICTED_USER_ADMIN", false, spec.User) && !hasRestrictedUserPriv { - return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_USER_ADMIN") + if sem.IsEnabled() { + hasRestrictedUser, err := userHasDynamicPrivilegeInternal(ctx, sqlExecutor, spec.User, "RESTRICTED_USER_ADMIN") + if err != nil { + return err + } + if hasRestrictedUser && !hasRestrictedUserPriv { + return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_USER_ADMIN") + } } } - exists, _, err := userExistsInternal(ctx, sqlExecutor, spec.User.Username, spec.User.Hostname) + exists, currentAuthPlugin, err := userExistsInternal(ctx, sqlExecutor, spec.User.Username, spec.User.Hostname) if err != nil { return err } @@ -1791,10 +1801,6 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) failedUsers = append(failedUsers, user) continue } - currentAuthPlugin, err := privilege.GetPrivilegeManager(e.Ctx()).GetAuthPlugin(spec.User.Username, spec.User.Hostname) - if err != nil { - return err - } type AuthTokenOptionHandler int const ( @@ -2307,7 +2313,14 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e // Because in TiDB SUPER can be used as a substitute for any dynamic privilege, this effectively means that // any user with SUPER requires a user with SUPER to be able to DROP the user. // We also allow RESTRICTED_USER_ADMIN to count for simplicity. - if checker.RequestDynamicVerificationWithUser("SYSTEM_USER", false, user) && !(hasSystemUserPriv || hasRestrictedUserPriv) { + hasSystemUser, err := userHasDynamicPrivilegeInternal(internalCtx, sqlExecutor, user, "SYSTEM_USER") + if err != nil { + if _, err := sqlExecutor.ExecuteInternal(internalCtx, "rollback"); err != nil { + return err + } + return err + } + if hasSystemUser && !(hasSystemUserPriv || hasRestrictedUserPriv) { if _, err := sqlExecutor.ExecuteInternal(internalCtx, "rollback"); err != nil { return err } @@ -2484,6 +2497,28 @@ func userExistsInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, na return rows > 0, authPlugin, err } +// userHasDynamicPrivilegeInternal checks if a user has a specific dynamic privilege by querying the database directly. +// This avoids loading the user into memory through ensureActiveUser. +func userHasDynamicPrivilegeInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, user *auth.UserIdentity, privName string) (bool, error) { + sql := new(strings.Builder) + sqlescape.MustFormatSQL(sql, `SELECT * FROM %n.%n WHERE User=%? AND Host=%? AND Priv=%?;`, mysql.SystemDB, "global_grants", user.Username, strings.ToLower(user.Hostname), privName) + recordSet, err := sqlExecutor.ExecuteInternal(ctx, sql.String()) + if err != nil { + return false, err + } + req := recordSet.NewChunk(nil) + err = recordSet.Next(ctx, req) + var rows = 0 + if err == nil { + rows = req.NumRows() + } + errClose := recordSet.Close() + if errClose != nil { + return false, errClose + } + return rows > 0, nil +} + func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnPrivilege) sysSession, err := e.GetSysSession() @@ -2525,7 +2560,7 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error return exeerrors.ErrDBaccessDenied.GenWithStackByArgs(currUser.Username, currUser.Hostname, "mysql") } } - exists, _, err := userExistsInternal(ctx, sqlExecutor, u, h) + exists, authplugin, err := userExistsInternal(ctx, sqlExecutor, u, h) if err != nil { return err } @@ -2539,11 +2574,6 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error } disableSandboxMode = true } - - authplugin, err := privilege.GetPrivilegeManager(e.Ctx()).GetAuthPlugin(u, h) - if err != nil { - return err - } if e.isValidatePasswordEnabled() { if err := pwdValidator.ValidatePassword(e.Ctx().GetSessionVars(), s.Password); err != nil { return err From 761465f4326e48c709d9d169960606ba8b839942 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 14:11:25 +0800 Subject: [PATCH 12/17] recover the 'active users' grafana --- pkg/metrics/grafana/tidb.json | 112 +++++++++++++++++++++++++++++++++- 1 file changed, 111 insertions(+), 1 deletion(-) diff --git a/pkg/metrics/grafana/tidb.json b/pkg/metrics/grafana/tidb.json index 551dcfab90fd3..e9929096373dc 100644 --- a/pkg/metrics/grafana/tidb.json +++ b/pkg/metrics/grafana/tidb.json @@ -4314,6 +4314,116 @@ "points": false, "stack": false, "steppedLine": false + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total count of active users.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 73 + }, + "hiddenSeries": false, + "id": 23763574014, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.17", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tidb_server_active_users{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "H" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Active Users", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:264", + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:265", + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -25563,4 +25673,4 @@ "title": "Test-Cluster-TiDB", "uid": "000000011", "version": 1 -} \ No newline at end of file +} From cdc5a8a9d54cd5cbc167dcb1e11890466661bc47 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 14:20:15 +0800 Subject: [PATCH 13/17] fix issues found by codex GPT 5.2 --- pkg/privilege/privileges/cache.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index 9ab17e1dc54d6..fb52476cb6496 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -1238,13 +1238,13 @@ func (p *MySQLPrivilege) decodeRoleEdgesTable(row chunk.Row, fs []*resolve.Resul for i, f := range fs { switch f.ColumnAsName.L { case "from_host": - fromHost = row.GetString(i) + fromHost = strings.Clone(row.GetString(i)) case "from_user": - fromUser = row.GetString(i) + fromUser = strings.Clone(row.GetString(i)) case "to_host": - toHost = row.GetString(i) + toHost = strings.Clone(row.GetString(i)) case "to_user": - toUser = row.GetString(i) + toUser = strings.Clone(row.GetString(i)) } } fromKey := auth.RoleIdentity{Username: fromUser, Hostname: fromHost} @@ -2140,6 +2140,7 @@ func (h *Handle) Get() *MySQLPrivilege { // UpdateAll loads all the users' privilege info from kv storage. func (h *Handle) UpdateAll() error { priv := newMySQLPrivilege() + priv.globalVars = h.globalVars res, err := h.sctx.Get() if err != nil { return errors.Trace(err) From b33fd9926c486c2641d55ebf81ba1aeb105ced65 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 15:03:47 +0800 Subject: [PATCH 14/17] fix CI TestPluginUserModification --- pkg/executor/simple.go | 43 +++--------------------------------------- 1 file changed, 3 insertions(+), 40 deletions(-) diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 562b386e81766..99c981f2bdfba 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -1774,19 +1774,11 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) if !(hasCreateUserPriv || hasSystemSchemaPriv) { return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER") } - hasSystemUser, err := userHasDynamicPrivilegeInternal(ctx, sqlExecutor, spec.User, "SYSTEM_USER") - if err != nil { - return err - } - if hasSystemUser && !(hasSystemUserPriv || hasRestrictedUserPriv) { + if !(hasSystemUserPriv || hasRestrictedUserPriv) && checker.RequestDynamicVerificationWithUser("SYSTEM_USER", false, spec.User) { return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("SYSTEM_USER or SUPER") } if sem.IsEnabled() { - hasRestrictedUser, err := userHasDynamicPrivilegeInternal(ctx, sqlExecutor, spec.User, "RESTRICTED_USER_ADMIN") - if err != nil { - return err - } - if hasRestrictedUser && !hasRestrictedUserPriv { + if !hasRestrictedUserPriv && checker.RequestDynamicVerificationWithUser("RESTRICTED_USER_ADMIN", false, spec.User) { return plannererrors.ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_USER_ADMIN") } } @@ -2313,14 +2305,7 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e // Because in TiDB SUPER can be used as a substitute for any dynamic privilege, this effectively means that // any user with SUPER requires a user with SUPER to be able to DROP the user. // We also allow RESTRICTED_USER_ADMIN to count for simplicity. - hasSystemUser, err := userHasDynamicPrivilegeInternal(internalCtx, sqlExecutor, user, "SYSTEM_USER") - if err != nil { - if _, err := sqlExecutor.ExecuteInternal(internalCtx, "rollback"); err != nil { - return err - } - return err - } - if hasSystemUser && !(hasSystemUserPriv || hasRestrictedUserPriv) { + if !(hasSystemUserPriv || hasRestrictedUserPriv) && checker.RequestDynamicVerificationWithUser("SYSTEM_USER", false, user) { if _, err := sqlExecutor.ExecuteInternal(internalCtx, "rollback"); err != nil { return err } @@ -2497,28 +2482,6 @@ func userExistsInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, na return rows > 0, authPlugin, err } -// userHasDynamicPrivilegeInternal checks if a user has a specific dynamic privilege by querying the database directly. -// This avoids loading the user into memory through ensureActiveUser. -func userHasDynamicPrivilegeInternal(ctx context.Context, sqlExecutor sqlexec.SQLExecutor, user *auth.UserIdentity, privName string) (bool, error) { - sql := new(strings.Builder) - sqlescape.MustFormatSQL(sql, `SELECT * FROM %n.%n WHERE User=%? AND Host=%? AND Priv=%?;`, mysql.SystemDB, "global_grants", user.Username, strings.ToLower(user.Hostname), privName) - recordSet, err := sqlExecutor.ExecuteInternal(ctx, sql.String()) - if err != nil { - return false, err - } - req := recordSet.NewChunk(nil) - err = recordSet.Next(ctx, req) - var rows = 0 - if err == nil { - rows = req.NumRows() - } - errClose := recordSet.Close() - if errClose != nil { - return false, errClose - } - return rows > 0, nil -} - func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnPrivilege) sysSession, err := e.GetSysSession() From 19b0be9903a1b50878f2c660740f8bdd512c4243 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 15:31:06 +0800 Subject: [PATCH 15/17] fix 'set password for user' call ensureActiveUser --- pkg/executor/simple.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 99c981f2bdfba..5188e10f826d7 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -2516,7 +2516,6 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error h = s.User.Hostname checker := privilege.GetPrivilegeManager(e.Ctx()) - checker.MatchIdentity(u, h, false) activeRoles := e.Ctx().GetSessionVars().ActiveRoles if checker != nil && !checker.RequestVerification(activeRoles, "", "", "", mysql.SuperPriv) { currUser := e.Ctx().GetSessionVars().User From 6dfdf937a13c65b14ab74c8d4b8c93dd481a29ac Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 16:40:22 +0800 Subject: [PATCH 16/17] fix CI --- pkg/extension/auth_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/extension/auth_test.go b/pkg/extension/auth_test.go index 1aa2917ba6545..4098eae81923d 100644 --- a/pkg/extension/auth_test.go +++ b/pkg/extension/auth_test.go @@ -235,7 +235,7 @@ func TestAuthPlugin(t *testing.T) { // Should authenticate using plugin impl. p.AssertNumberOfCalls(t, "AuthenticateUser", 2) p.AssertCalled(t, "ValidateAuthString", "encodedpassword") - p.AssertNumberOfCalls(t, "ValidateAuthString", 4) + p.AssertNumberOfCalls(t, "ValidateAuthString", 3) // Change password should work using ALTER USER statement. tk.MustExec("alter user 'u2'@'localhost' identified with 'authentication_test_plugin' by 'anotherrawpassword'") From 4705895e7eb36e55a4fabaa2eea2be19d3333e01 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Jan 2026 18:20:38 +0800 Subject: [PATCH 17/17] fix CI --- pkg/executor/infoschema_reader_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/pkg/executor/infoschema_reader_test.go b/pkg/executor/infoschema_reader_test.go index 0afb2b077458f..6ab3991c1371b 100644 --- a/pkg/executor/infoschema_reader_test.go +++ b/pkg/executor/infoschema_reader_test.go @@ -711,8 +711,6 @@ func TestIndexUsageTable(t *testing.T) { where TABLE_SCHEMA = 'test' and TABLE_NAME = 'idt2' and INDEX_NAME = 'idx_4';`).Check( testkit.RowsWithSep("|", "test|idt2|idx_4")) - tk.MustQuery(`select count(*) from information_schema.tidb_index_usage;`).Check( - testkit.RowsWithSep("|", "81")) tk.MustQuery(`select TABLE_SCHEMA, TABLE_NAME, INDEX_NAME from information_schema.tidb_index_usage where TABLE_SCHEMA = 'test1';`).Check(testkit.Rows())