diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index adf850f6a7..cb76a2fe6b 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -1902,7 +1902,7 @@ func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { if err != nil { return err } - do.privHandle = privileges.NewHandle(sctx.GetRestrictedSQLExecutor()) + do.privHandle = privileges.NewHandle(sctx.GetRestrictedSQLExecutor(), sctx.GetSessionVars().GlobalVarsAccessor) var watchCh clientv3.WatchChan duration := 5 * time.Minute diff --git a/pkg/executor/grant.go b/pkg/executor/grant.go index d39f99c319..f941d200f2 100644 --- a/pkg/executor/grant.go +++ b/pkg/executor/grant.go @@ -155,6 +155,10 @@ func (e *GrantExec) Next(ctx context.Context, _ *chunk.Chunk) error { return err } + defaultAuthPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.DefaultAuthPlugin) + if err != nil { + return err + } // Check which user is not exist. for _, user := range e.Users { exists, err := userExists(ctx, e.Ctx(), user.User.Username, user.User.Hostname) @@ -169,10 +173,7 @@ func (e *GrantExec) Next(ctx context.Context, _ *chunk.Chunk) error { // It is required for compatibility with 5.7 but removed from 8.0 // since it results in a massive security issue: // spelling errors will create users with no passwords. - authPlugin, err := e.Ctx().GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.DefaultAuthPlugin) - if err != nil { - return err - } + authPlugin := defaultAuthPlugin if user.AuthOpt != nil && user.AuthOpt.AuthPlugin != "" { authPlugin = user.AuthOpt.AuthPlugin } @@ -181,7 +182,7 @@ func (e *GrantExec) Next(ctx context.Context, _ *chunk.Chunk) error { return exeerrors.ErrPluginIsNotLoaded.GenWithStackByArgs(extErr.Error()) } authPluginImpl := extensions.GetAuthPlugins()[authPlugin] - pwd, ok := encodePassword(user, authPluginImpl) + pwd, ok := encodePasswordWithPlugin(*user, authPluginImpl, defaultAuthPlugin) if !ok { return errors.Trace(exeerrors.ErrPasswordFormat) } diff --git a/pkg/executor/simple.go b/pkg/executor/simple.go index 76b7799ed1..ae4e6517e8 100644 --- a/pkg/executor/simple.go +++ b/pkg/executor/simple.go @@ -1184,7 +1184,7 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm } } - pwd, ok := encodePassword(spec, pluginImpl) + pwd, ok := encodePasswordWithPlugin(*spec, pluginImpl, defaultAuthPlugin) if !ok { return errors.Trace(exeerrors.ErrPasswordFormat) } @@ -1856,7 +1856,8 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) return err } } - pwd, ok := encodePassword(spec, authPluginImpl) + // we have assigned the currentAuthPlugin to spec.AuthOpt.AuthPlugin if the latter is empty, so keep the incomming argument defaultPlugin empty is ok. + pwd, ok := encodePasswordWithPlugin(*spec, authPluginImpl, "") if !ok { return errors.Trace(exeerrors.ErrPasswordFormat) } diff --git a/pkg/executor/utils.go b/pkg/executor/utils.go index abd2cb7c06..d29ac17086 100644 --- a/pkg/executor/utils.go +++ b/pkg/executor/utils.go @@ -21,6 +21,8 @@ import ( "github.com/pingcap/tidb/pkg/extension" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/parser/mysql" ) var ( @@ -103,8 +105,8 @@ func (b *batchRetrieverHelper) nextBatch(retrieveRange func(start, end int) erro return nil } -// encodePassword encodes the password for the user. It invokes the auth plugin if it is available. -func encodePassword(u *ast.UserSpec, authPlugin *extension.AuthPlugin) (string, bool) { +// encodePasswordWithPlugin encodes the password for the user. It invokes the auth plugin if it is available. +func encodePasswordWithPlugin(u ast.UserSpec, authPlugin *extension.AuthPlugin, defaultPlugin string) (string, bool) { if u.AuthOpt == nil { return "", true } @@ -119,7 +121,66 @@ func encodePassword(u *ast.UserSpec, authPlugin *extension.AuthPlugin) (string, } return "", false } - return u.EncodedPassword() + return encodedPassword(&u, defaultPlugin) +} + +// encodedPassword returns the encoded password (which is the real data mysql.user). +// The boolean value indicates input's password format is legal or not. +func encodedPassword(n *ast.UserSpec, defaultPlugin string) (string, bool) { + if n.AuthOpt == nil { + return "", true + } + + opt := n.AuthOpt + authPlugin := opt.AuthPlugin + if authPlugin == "" { + authPlugin = defaultPlugin + } + if opt.ByAuthString { + switch authPlugin { + case mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password: + return auth.NewHashPassword(opt.AuthString, authPlugin), true + case mysql.AuthSocket: + return "", true + default: + return auth.EncodePassword(opt.AuthString), true + } + } + + // store the LDAP dn directly in the password field + switch authPlugin { + case mysql.AuthLDAPSimple, mysql.AuthLDAPSASL: + // TODO: validate the HashString to be a `dn` for LDAP + // It seems fine to not validate here, and LDAP server will give an error when the client'll try to login this user. + // The percona server implementation doesn't have a validation for this HashString. + // However, returning an error for obvious wrong format is more friendly. + return opt.HashString, true + } + + // In case we have 'IDENTIFIED WITH ' but no 'BY ' to set an empty password. + if opt.HashString == "" { + return opt.HashString, true + } + + // Not a legal password string. + switch authPlugin { + case mysql.AuthCachingSha2Password: + if len(opt.HashString) != mysql.SHAPWDHashLen { + return "", false + } + case mysql.AuthTiDBSM3Password: + if len(opt.HashString) != mysql.SM3PWDHashLen { + return "", false + } + case "", mysql.AuthNativePassword: + if len(opt.HashString) != (mysql.PWDHashLen+1) || !strings.HasPrefix(opt.HashString, "*") { + return "", false + } + case mysql.AuthSocket: + default: + return "", false + } + return opt.HashString, true } var taskPool = sync.Pool{ diff --git a/pkg/executor/utils_test.go b/pkg/executor/utils_test.go index 0bcfc2d843..67f792b757 100644 --- a/pkg/executor/utils_test.go +++ b/pkg/executor/utils_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/extension" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/require" @@ -165,17 +166,17 @@ func TestEncodePasswordWithPlugin(t *testing.T) { } u.AuthOpt.ByAuthString = false - _, ok := encodePassword(u, p) + _, ok := encodePasswordWithPlugin(*u, p, "") require.False(t, ok) u.AuthOpt.AuthString = "xxx" u.AuthOpt.ByAuthString = true - pwd, ok := encodePassword(u, p) + pwd, ok := encodePasswordWithPlugin(*u, p, "") require.True(t, ok) require.Equal(t, "xxxxxxx", pwd) u.AuthOpt = nil - pwd, ok = encodePassword(u, p) + pwd, ok = encodePasswordWithPlugin(*u, p, "") require.True(t, ok) require.Equal(t, "", pwd) } @@ -271,3 +272,41 @@ func TestGoPool(t *testing.T) { require.Equal(t, []int{1, 2, 3, 4}, list) }) } + +func TestEncodedPassword(t *testing.T) { + hashString := "*3D56A309CD04FA2EEF181462E59011F075C89548" + hashCachingString := "0123456789012345678901234567890123456789012345678901234567890123456789" + u := ast.UserSpec{ + User: &auth.UserIdentity{ + Username: "test", + }, + AuthOpt: &ast.AuthOption{ + ByAuthString: false, + AuthString: "xxx", + HashString: hashString, + }, + } + pwd, ok := encodedPassword(&u, "") + require.True(t, ok) + require.Equal(t, u.AuthOpt.HashString, pwd) + + u.AuthOpt.HashString = "not-good-password-format" + _, ok = encodedPassword(&u, "") + require.False(t, ok) + + u.AuthOpt.ByAuthString = true + // mysql_native_password + pwd, ok = encodedPassword(&u, "") + require.True(t, ok) + require.Equal(t, hashString, pwd) + // caching_sha2_password + u.AuthOpt.HashString = hashCachingString + pwd, ok = encodedPassword(&u, mysql.AuthCachingSha2Password) + require.True(t, ok) + require.Len(t, pwd, mysql.SHAPWDHashLen) + + u.AuthOpt.AuthString = "" + pwd, ok = encodedPassword(&u, "") + require.True(t, ok) + require.Equal(t, "", pwd) +} diff --git a/pkg/parser/ast/BUILD.bazel b/pkg/parser/ast/BUILD.bazel index 14898f8435..2e763a8abf 100644 --- a/pkg/parser/ast/BUILD.bazel +++ b/pkg/parser/ast/BUILD.bazel @@ -52,7 +52,6 @@ go_test( shard_count = 50, deps = [ "//pkg/parser", - "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/format", "//pkg/parser/mysql", diff --git a/pkg/parser/ast/misc.go b/pkg/parser/ast/misc.go index 6aac75c2d7..349f7c3680 100644 --- a/pkg/parser/ast/misc.go +++ b/pkg/parser/ast/misc.go @@ -1523,61 +1523,6 @@ func (n *UserSpec) SecurityString() string { return n.User.String() } -// EncodedPassword returns the encoded password (which is the real data mysql.user). -// The boolean value indicates input's password format is legal or not. -func (n *UserSpec) EncodedPassword() (string, bool) { - if n.AuthOpt == nil { - return "", true - } - - opt := n.AuthOpt - if opt.ByAuthString { - switch opt.AuthPlugin { - case mysql.AuthCachingSha2Password, mysql.AuthTiDBSM3Password: - return auth.NewHashPassword(opt.AuthString, opt.AuthPlugin), true - case mysql.AuthSocket: - return "", true - default: - return auth.EncodePassword(opt.AuthString), true - } - } - - // store the LDAP dn directly in the password field - switch opt.AuthPlugin { - case mysql.AuthLDAPSimple, mysql.AuthLDAPSASL: - // TODO: validate the HashString to be a `dn` for LDAP - // It seems fine to not validate here, and LDAP server will give an error when the client'll try to login this user. - // The percona server implementation doesn't have a validation for this HashString. - // However, returning an error for obvious wrong format is more friendly. - return opt.HashString, true - } - - // In case we have 'IDENTIFIED WITH ' but no 'BY ' to set an empty password. - if opt.HashString == "" { - return opt.HashString, true - } - - // Not a legal password string. - switch opt.AuthPlugin { - case mysql.AuthCachingSha2Password: - if len(opt.HashString) != mysql.SHAPWDHashLen { - return "", false - } - case mysql.AuthTiDBSM3Password: - if len(opt.HashString) != mysql.SM3PWDHashLen { - return "", false - } - case "", mysql.AuthNativePassword: - if len(opt.HashString) != (mysql.PWDHashLen+1) || !strings.HasPrefix(opt.HashString, "*") { - return "", false - } - case mysql.AuthSocket: - default: - return "", false - } - return opt.HashString, true -} - type AuthTokenOrTLSOption struct { Type AuthTokenOrTLSOptionType Value string diff --git a/pkg/parser/ast/misc_test.go b/pkg/parser/ast/misc_test.go index 1c32597164..1062058322 100644 --- a/pkg/parser/ast/misc_test.go +++ b/pkg/parser/ast/misc_test.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/stretchr/testify/require" ) @@ -165,37 +164,6 @@ func TestSensitiveStatement(t *testing.T) { } } -func TestUserSpec(t *testing.T) { - hashString := "*3D56A309CD04FA2EEF181462E59011F075C89548" - u := ast.UserSpec{ - User: &auth.UserIdentity{ - Username: "test", - }, - AuthOpt: &ast.AuthOption{ - ByAuthString: false, - AuthString: "xxx", - HashString: hashString, - }, - } - pwd, ok := u.EncodedPassword() - require.True(t, ok) - require.Equal(t, u.AuthOpt.HashString, pwd) - - u.AuthOpt.HashString = "not-good-password-format" - _, ok = u.EncodedPassword() - require.False(t, ok) - - u.AuthOpt.ByAuthString = true - pwd, ok = u.EncodedPassword() - require.True(t, ok) - require.Equal(t, hashString, pwd) - - u.AuthOpt.AuthString = "" - pwd, ok = u.EncodedPassword() - require.True(t, ok) - require.Equal(t, "", pwd) -} - func TestTableOptimizerHintRestore(t *testing.T) { testCases := []NodeRestoreTestCase{ {"USE_INDEX(t1 c1)", "USE_INDEX(`t1` `c1`)"}, diff --git a/pkg/privilege/privilege.go b/pkg/privilege/privilege.go index f421c3f657..e84a894a3d 100644 --- a/pkg/privilege/privilege.go +++ b/pkg/privilege/privilege.go @@ -116,9 +116,6 @@ type Manager interface { // GetAuthPluginForConnection gets the authentication plugin used in connection establishment. GetAuthPluginForConnection(ctx context.Context, user, host string) (string, error) - - // GetAuthPlugin gets the authentication plugin for the account identified by the user and host - GetAuthPlugin(ctx context.Context, user, host string) (string, error) } const key keyType = 0 diff --git a/pkg/privilege/privileges/cache.go b/pkg/privilege/privileges/cache.go index acdef9609b..0fcc5c4c18 100644 --- a/pkg/privilege/privileges/cache.go +++ b/pkg/privilege/privileges/cache.go @@ -273,6 +273,8 @@ type immutable struct { globalPriv []globalPrivRecord dynamicPriv []dynamicPrivRecord roleGraph map[string]roleGraphEdgesTable + + globalVars variable.GlobalVarAccessor } type extended struct { @@ -916,6 +918,16 @@ func (record *baseRecord) assignUserOrHost(row chunk.Row, i int, f *resolve.Resu func (p *immutable) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) error { var value UserRecord + defaultAuthPlugin := "" + if p.globalVars != nil { + val, err := p.globalVars.GetGlobalSysVar(variable.DefaultAuthPlugin) + if err != nil { + defaultAuthPlugin = val + } + } + if defaultAuthPlugin == "" { + defaultAuthPlugin = mysql.AuthNativePassword + } for i, f := range fs { switch { case f.ColumnAsName.L == "authentication_string": @@ -928,7 +940,7 @@ func (p *immutable) decodeUserTableRow(row chunk.Row, fs []*resolve.ResultField) if row.GetString(i) != "" { value.AuthPlugin = row.GetString(i) } else { - value.AuthPlugin = mysql.AuthNativePassword + value.AuthPlugin = defaultAuthPlugin } case f.ColumnAsName.L == "token_issuer": value.AuthTokenIssuer = row.GetString(i) @@ -1884,13 +1896,14 @@ type Handle struct { // Only load the active user's data to save memory // username => struct{} activeUsers sync.Map + + globalVars variable.GlobalVarAccessor } // NewHandle returns a Handle. -func NewHandle(sctx sqlexec.RestrictedSQLExecutor) *Handle { +func NewHandle(sctx sqlexec.RestrictedSQLExecutor, globalVars variable.GlobalVarAccessor) *Handle { var priv MySQLPrivilege - ret := &Handle{} - ret.sctx = sctx + ret := &Handle{sctx: sctx, globalVars: globalVars} ret.priv.Store(&priv) return ret } @@ -1907,6 +1920,7 @@ func (h *Handle) ensureActiveUser(ctx context.Context, user string) error { return nil } var data immutable + data.globalVars = h.globalVars userList, err := data.loadSomeUsers(h.sctx, user) if err != nil { return errors.Trace(err) @@ -1943,6 +1957,7 @@ func (h *Handle) UpdateAll() error { }) var priv immutable + priv.globalVars = h.globalVars userList, err := priv.loadSomeUsers(h.sctx, userList...) if err != nil { return err @@ -1965,6 +1980,7 @@ func (h *Handle) Update(userList []string) error { } var priv immutable + priv.globalVars = h.globalVars userList, err := priv.loadSomeUsers(h.sctx, userList...) if err != nil { return err diff --git a/pkg/privilege/privileges/privileges.go b/pkg/privilege/privileges/privileges.go index 0145aa5263..30eebd611e 100644 --- a/pkg/privilege/privileges/privileges.go +++ b/pkg/privilege/privileges/privileges.go @@ -349,24 +349,6 @@ func (p *UserPrivileges) GetAuthPluginForConnection(ctx context.Context, user, h return "", errors.New("Failed to get plugin for user") } -// GetAuthPlugin gets the authentication plugin for the account identified by the user and host -func (p *UserPrivileges) GetAuthPlugin(ctx context.Context, user, host string) (string, error) { - if SkipWithGrant { - return mysql.AuthNativePassword, nil - } - - terror.Log(p.Handle.ensureActiveUser(ctx, user)) - mysqlPriv := p.Handle.Get() - record := mysqlPriv.connectionVerification(user, host) - if record == nil { - return "", errors.New("Failed to get user record") - } - if !p.isValidHash(record) { - return "", errors.New("Failed to get plugin for user") - } - return record.AuthPlugin, nil -} - // MatchIdentity implements the Manager interface. func (p *UserPrivileges) MatchIdentity(ctx context.Context, user, host string, skipNameResolve bool) (u string, h string, success bool) { if SkipWithGrant { diff --git a/pkg/privilege/privileges/privileges_test.go b/pkg/privilege/privileges/privileges_test.go index 7a0da0a20d..8ec8e21712 100644 --- a/pkg/privilege/privileges/privileges_test.go +++ b/pkg/privilege/privileges/privileges_test.go @@ -1904,7 +1904,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) diff --git a/pkg/server/conn.go b/pkg/server/conn.go index 6604d8c5f5..f7a45129d4 100644 --- a/pkg/server/conn.go +++ b/pkg/server/conn.go @@ -886,15 +886,13 @@ func (cc *clientConn) checkAuthPlugin(ctx context.Context, resp *handshake.Respo // No user plugin set, assuming MySQL Native Password // This happens if the account doesn't exist or if the account doesn't have // a password set. - if resp.AuthPlugin != mysql.AuthNativePassword { - if resp.Capability&mysql.ClientPluginAuth > 0 { - resp.AuthPlugin = mysql.AuthNativePassword - authData, err := cc.authSwitchRequest(ctx, mysql.AuthNativePassword) - if err != nil { - return nil, err - } - return authData, nil + if resp.AuthPlugin != mysql.AuthNativePassword && resp.Capability&mysql.ClientPluginAuth > 0 { + resp.AuthPlugin = mysql.AuthNativePassword + authData, err := cc.authSwitchRequest(ctx, mysql.AuthNativePassword) + if err != nil { + return nil, err } + return authData, nil } return nil, nil } diff --git a/tests/integrationtest/r/executor/simple.result b/tests/integrationtest/r/executor/simple.result index 377bc8534d..5a4427a11b 100644 --- a/tests/integrationtest/r/executor/simple.result +++ b/tests/integrationtest/r/executor/simple.result @@ -491,6 +491,13 @@ CREATE USER 'default_sha2_role'@'%' IDENTIFIED WITH 'caching_sha2_password' AS ' select plugin from mysql.user where user = 'default_sha2_role'; plugin caching_sha2_password +drop user default_sha2_user; +drop user native_plugin_user; +create user default_sha2_user identified by '1234'; +create user native_plugin_user identified with 'mysql_native_password' by '1234'; +select count(distinct authentication_string) from mysql.user where user = 'default_sha2_user' or user = 'native_plugin_user'; +count(distinct authentication_string) +2 alter user default_sha2_user identified with 'tidb_sm3_password'; show create user default_sha2_user; CREATE USER for default_sha2_user@% @@ -498,20 +505,42 @@ CREATE USER 'default_sha2_user'@'%' IDENTIFIED WITH 'tidb_sm3_password' AS '' RE select plugin from mysql.user where user = 'default_sha2_user'; plugin tidb_sm3_password -alter user default_sha2_user identified with 'authentication_ldap_simple'; -show create user default_sha2_user; -CREATE USER for default_sha2_user@% -CREATE USER 'default_sha2_user'@'%' IDENTIFIED WITH 'authentication_ldap_simple' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK PASSWORD HISTORY DEFAULT PASSWORD REUSE INTERVAL DEFAULT +drop user default_sha2_user; +create user default_sha2_user identified by '1234'; +set password for default_sha2_user = '12345'; +alter user default_sha2_user identified by '123456'; select plugin from mysql.user where user = 'default_sha2_user'; plugin -authentication_ldap_simple -alter user default_sha2_user identified with 'authentication_ldap_sasl'; -show create user default_sha2_user; -CREATE USER for default_sha2_user@% -CREATE USER 'default_sha2_user'@'%' IDENTIFIED WITH 'authentication_ldap_sasl' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK PASSWORD HISTORY DEFAULT PASSWORD REUSE INTERVAL DEFAULT +caching_sha2_password +alter user default_sha2_user identified with 'mysql_native_password' by '123456'; select plugin from mysql.user where user = 'default_sha2_user'; plugin -authentication_ldap_sasl +mysql_native_password +alter user default_sha2_user identified with 'caching_sha2_password'; +select plugin, length(authentication_string) from mysql.user where user = 'default_sha2_user'; +plugin length(authentication_string) +caching_sha2_password 0 +set sql_mode = ''; +select @@sql_mode; +@@sql_mode + +select user, host, plugin from mysql.user where user = 'non_exist_user'; +user host plugin +grant select on test.* to non_exist_user; +select user, host, plugin from mysql.user where user = 'non_exist_user'; +user host plugin +non_exist_user % caching_sha2_password +set @@sql_mode = default; +alter user non_exist_user identified with 'mysql_native_password'; +show create user non_exist_user; +CREATE USER for non_exist_user@% +CREATE USER 'non_exist_user'@'%' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK PASSWORD HISTORY DEFAULT PASSWORD REUSE INTERVAL DEFAULT +update mysql.user set plugin = '' where user = 'non_exist_user'; +flush privileges; +show create user non_exist_user; +CREATE USER for non_exist_user@% +CREATE USER 'non_exist_user'@'%' IDENTIFIED WITH 'caching_sha2_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK PASSWORD HISTORY DEFAULT PASSWORD REUSE INTERVAL DEFAULT +drop user non_exist_user; drop user default_sm3_user; drop user default_sha2_user; drop user native_plugin_user; diff --git a/tests/integrationtest/t/executor/simple.test b/tests/integrationtest/t/executor/simple.test index a34a87195d..89d6b31c33 100644 --- a/tests/integrationtest/t/executor/simple.test +++ b/tests/integrationtest/t/executor/simple.test @@ -514,18 +514,46 @@ select plugin from mysql.user where user = 'native_plugin_user'; show create user default_sha2_role; select plugin from mysql.user where user = 'default_sha2_role'; +# default_sha2_user and native_plugin_user should encode the password with different way. +drop user default_sha2_user; +drop user native_plugin_user; +create user default_sha2_user identified by '1234'; +create user native_plugin_user identified with 'mysql_native_password' by '1234'; +select count(distinct authentication_string) from mysql.user where user = 'default_sha2_user' or user = 'native_plugin_user'; + alter user default_sha2_user identified with 'tidb_sm3_password'; show create user default_sha2_user; select plugin from mysql.user where user = 'default_sha2_user'; +drop user default_sha2_user; -alter user default_sha2_user identified with 'authentication_ldap_simple'; -show create user default_sha2_user; +create user default_sha2_user identified by '1234'; +set password for default_sha2_user = '12345'; +alter user default_sha2_user identified by '123456'; select plugin from mysql.user where user = 'default_sha2_user'; - -alter user default_sha2_user identified with 'authentication_ldap_sasl'; -show create user default_sha2_user; +alter user default_sha2_user identified with 'mysql_native_password' by '123456'; select plugin from mysql.user where user = 'default_sha2_user'; +alter user default_sha2_user identified with 'caching_sha2_password'; +# the authentication_string should be empty +select plugin, length(authentication_string) from mysql.user where user = 'default_sha2_user'; +# test GRANT create default user + +set sql_mode = ''; +select @@sql_mode; +select user, host, plugin from mysql.user where user = 'non_exist_user'; +grant select on test.* to non_exist_user; +select user, host, plugin from mysql.user where user = 'non_exist_user'; +set @@sql_mode = default; + +# test SHOW CREATE USER + +alter user non_exist_user identified with 'mysql_native_password'; +show create user non_exist_user; +update mysql.user set plugin = '' where user = 'non_exist_user'; +flush privileges; +show create user non_exist_user; + +drop user non_exist_user; drop user default_sm3_user; drop user default_sha2_user; drop user native_plugin_user;