diff --git a/executor/simple.go b/executor/simple.go index 7270f12aecdd0..d1e5181e2f439 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -136,6 +136,8 @@ func (e *SimpleExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { err = e.executeAlterUser(x) case *ast.DropUserStmt: err = e.executeDropUser(x) + case *ast.RenameUserStmt: + err = e.executeRenameUser(x) case *ast.SetPwdStmt: err = e.executeSetPwd(x) case *ast.KillStmt: @@ -1026,6 +1028,123 @@ func (e *SimpleExec) executeGrantRole(s *ast.GrantRoleStmt) error { return nil } +// Should cover same internal mysql.* tables as DROP USER, so this function is very similar +func (e *SimpleExec) executeRenameUser(s *ast.RenameUserStmt) error { + + var failedUser string + sysSession, err := e.getSysSession() + defer e.releaseSysSession(sysSession) + if err != nil { + return err + } + sqlExecutor := sysSession.(sqlexec.SQLExecutor) + + if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "begin"); err != nil { + return err + } + + for _, userToUser := range s.UserToUsers { + oldUser, newUser := userToUser.OldUser, userToUser.NewUser + exists, err := userExistsInternal(sqlExecutor, oldUser.Username, oldUser.Hostname) + if err != nil { + return err + } + if !exists { + failedUser = oldUser.String() + " TO " + newUser.String() + " old did not exist" + break + } + + exists, err = userExistsInternal(sqlExecutor, newUser.Username, newUser.Hostname) + if err != nil { + return err + } + if exists { + // MySQL reports the old user, even when the issue is the new user. + failedUser = oldUser.String() + " TO " + newUser.String() + " new did exist" + break + } + + if err = renameUserHostInSystemTable(sqlExecutor, mysql.UserTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.UserTable + " error" + break + } + + // rename privileges from mysql.global_priv + if err = renameUserHostInSystemTable(sqlExecutor, mysql.GlobalPrivTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.GlobalPrivTable + " error" + break + } + + // rename privileges from mysql.db + if err = renameUserHostInSystemTable(sqlExecutor, mysql.DBTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.DBTable + " error" + break + } + + // rename privileges from mysql.tables_priv + if err = renameUserHostInSystemTable(sqlExecutor, mysql.TablePrivTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.TablePrivTable + " error" + break + } + + // rename relationship from mysql.role_edges + if err = renameUserHostInSystemTable(sqlExecutor, mysql.RoleEdgeTable, "TO_USER", "TO_HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.RoleEdgeTable + " (to) error" + break + } + + if err = renameUserHostInSystemTable(sqlExecutor, mysql.RoleEdgeTable, "FROM_USER", "FROM_HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.RoleEdgeTable + " (from) error" + break + } + + // rename relationship from mysql.default_roles + if err = renameUserHostInSystemTable(sqlExecutor, mysql.DefaultRoleTable, "DEFAULT_ROLE_USER", "DEFAULT_ROLE_HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.DefaultRoleTable + " (default role user) error" + break + } + + if err = renameUserHostInSystemTable(sqlExecutor, mysql.DefaultRoleTable, "USER", "HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.DefaultRoleTable + " error" + break + } + + // rename relationship from mysql.global_grants + // TODO: add global_grants into the parser + if err = renameUserHostInSystemTable(sqlExecutor, "global_grants", "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " mysql.global_grants error" + break + } + + //TODO: need update columns_priv once we implement columns_priv functionality. + // When that is added, please refactor both executeRenameUser and executeDropUser to use an array of tables + // to loop over, so it is easier to maintain. + } + + if failedUser == "" { + if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "commit"); err != nil { + return err + } + } else { + if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "rollback"); err != nil { + return err + } + return ErrCannotUser.GenWithStackByArgs("RENAME USER", failedUser) + } + domain.GetDomain(e.ctx).NotifyUpdatePrivilege(e.ctx) + return nil +} + +func renameUserHostInSystemTable(sqlExecutor sqlexec.SQLExecutor, tableName, usernameColumn, hostColumn string, users *ast.UserToUser) error { + sql := new(strings.Builder) + sqlexec.MustFormatSQL(sql, `UPDATE %n.%n SET %n = %?, %n = %? WHERE %n = %? and %n = %?;`, + mysql.SystemDB, tableName, + usernameColumn, users.NewUser.Username, hostColumn, users.NewUser.Hostname, + usernameColumn, users.OldUser.Username, hostColumn, users.OldUser.Hostname) + _, err := sqlExecutor.ExecuteInternal(context.TODO(), sql.String()) + return err +} + func (e *SimpleExec) executeDropUser(s *ast.DropUserStmt) error { // Check privileges. // Check `CREATE USER` privilege. @@ -1181,6 +1300,27 @@ func userExists(ctx sessionctx.Context, name string, host string) (bool, error) return len(rows) > 0, nil } +// use the same internal executor to read within the same transaction, otherwise same as userExists +func userExistsInternal(sqlExecutor sqlexec.SQLExecutor, name string, host string) (bool, error) { + sql := new(strings.Builder) + sqlexec.MustFormatSQL(sql, `SELECT * FROM %n.%n WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, name, host) + recordSet, err := sqlExecutor.ExecuteInternal(context.TODO(), sql.String()) + if err != nil { + return false, err + } + req := recordSet.NewChunk() + err = recordSet.Next(context.TODO(), req) + var rows int = 0 + if err == nil { + rows = req.NumRows() + } + errClose := recordSet.Close() + if errClose != nil { + return false, errClose + } + return rows > 0, err +} + func (e *SimpleExec) executeSetPwd(s *ast.SetPwdStmt) error { var u, h string if s.User == nil { @@ -1389,7 +1529,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) (err error) { func (e *SimpleExec) autoNewTxn() bool { switch e.Statement.(type) { - case *ast.CreateUserStmt, *ast.AlterUserStmt, *ast.DropUserStmt: + case *ast.CreateUserStmt, *ast.AlterUserStmt, *ast.DropUserStmt, *ast.RenameUserStmt: return true } return false diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 921f1c99b34ec..56652983ff8f8 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1296,6 +1296,12 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", true}, }, }, + { + sql: "RENAME USER user1 to user1_tmp", + ans: []visitInfo{ + {mysql.CreateUserPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, + }, + }, } for _, tt := range tests { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 7dc2459dace33..ccab0a28cc863 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -641,7 +641,8 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { case *ast.BinlogStmt, *ast.FlushStmt, *ast.UseStmt, *ast.BRIEStmt, *ast.BeginStmt, *ast.CommitStmt, *ast.RollbackStmt, *ast.CreateUserStmt, *ast.SetPwdStmt, *ast.AlterInstanceStmt, *ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt, - *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt: + *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt, + *ast.RenameUserStmt: return b.buildSimple(node.(ast.StmtNode)) case ast.DDLNode: return b.buildDDL(ctx, x) @@ -2268,7 +2269,7 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { case *ast.AlterInstanceStmt: err := ErrSpecificAccessDenied.GenWithStack("SUPER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) - case *ast.AlterUserStmt: + case *ast.AlterUserStmt, *ast.RenameUserStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateUserPriv, "", "", "", err) case *ast.GrantStmt: diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 2f6cbef8af2cf..3917d822aa403 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1400,6 +1400,54 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { }, nil, nil) } +func (s *testPrivilegeSuite) TestRenameUser(c *C) { + rootSe := newSession(c, s.store, s.dbName) + mustExec(c, rootSe, "DROP USER IF EXISTS 'ru1'@'localhost'") + mustExec(c, rootSe, "DROP USER IF EXISTS ru3") + mustExec(c, rootSe, "DROP USER IF EXISTS ru6@localhost") + mustExec(c, rootSe, "CREATE USER 'ru1'@'localhost'") + mustExec(c, rootSe, "CREATE USER ru3") + mustExec(c, rootSe, "CREATE USER ru6@localhost") + se1 := newSession(c, s.store, s.dbName) + c.Assert(se1.Auth(&auth.UserIdentity{Username: "ru1", Hostname: "localhost"}, nil, nil), IsTrue) + + // Check privileges (need CREATE USER) + _, err := se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") + c.Assert(err, ErrorMatches, ".*Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation") + mustExec(c, rootSe, "GRANT UPDATE ON mysql.user TO 'ru1'@'localhost'") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") + c.Assert(err, ErrorMatches, ".*Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation") + mustExec(c, rootSe, "GRANT CREATE USER ON *.* TO 'ru1'@'localhost'") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") + c.Assert(err, IsNil) + + // Test a few single rename (both Username and Hostname) + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru4'@'%' TO 'ru3'@'localhost'") + c.Assert(err, IsNil) + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru3'@'localhost' TO 'ru3'@'%'") + c.Assert(err, IsNil) + // Including negative tests, i.e. non existing from user and existing to user + _, err = rootSe.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru1@localhost") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru3@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru4 TO ru5@localhost") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru4@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru3") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru3@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru5@localhost, ru4 TO ru7") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru4@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru5@localhost, ru6@localhost TO ru1@localhost") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru6@localhost.*") + + // Test multi rename, this is a full swap of ru3 and ru6, i.e. need to read its previous state in the same transaction. + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru3' TO 'ru3_tmp', ru6@localhost TO ru3, 'ru3_tmp' to ru6@localhost") + c.Assert(err, IsNil) + + // Cleanup + mustExec(c, rootSe, "DROP USER ru6@localhost") + mustExec(c, rootSe, "DROP USER ru3") + mustExec(c, rootSe, "DROP USER 'ru1'@'localhost'") +} + func (s *testPrivilegeSuite) TestSecurityEnhancedModeSysVars(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE USER svroot1, svroot2") diff --git a/session/session.go b/session/session.go index 902f35ca28e79..f116daf96dd04 100644 --- a/session/session.go +++ b/session/session.go @@ -2862,7 +2862,8 @@ func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) { switch stmt := execStmt.StmtNode.(type) { case *ast.CreateUserStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.SetPwdStmt, *ast.GrantStmt, *ast.RevokeStmt, *ast.AlterTableStmt, *ast.CreateDatabaseStmt, *ast.CreateIndexStmt, *ast.CreateTableStmt, - *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt: + *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt, + *ast.RenameUserStmt: user := vars.User schemaVersion := vars.GetInfoSchema().SchemaMetaVersion() if ss, ok := execStmt.StmtNode.(ast.SensitiveStmtNode); ok { diff --git a/session/tidb.go b/session/tidb.go index 85732b457f7a6..583c5074e6805 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -239,7 +239,7 @@ func autoCommitAfterStmt(ctx context.Context, se *session, meetsErr error, sql s sessVars := se.sessionVars if meetsErr != nil { if !sessVars.InTxn() { - logutil.BgLogger().Info("rollbackTxn for ddl/autocommit failed") + logutil.BgLogger().Info("rollbackTxn called due to ddl/autocommit failure") se.RollbackTxn(ctx) recordAbortTxnDuration(sessVars) } else if se.txn.Valid() && se.txn.IsPessimistic() && executor.ErrDeadlock.Equal(meetsErr) {