*: fix a bug for default_authentication_plugin (#57391)
ref pingcap/tidb#54138
This commit is contained in:
@ -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
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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 <plugin>' but no 'BY <password>' 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{
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -52,7 +52,6 @@ go_test(
|
||||
shard_count = 50,
|
||||
deps = [
|
||||
"//pkg/parser",
|
||||
"//pkg/parser/auth",
|
||||
"//pkg/parser/charset",
|
||||
"//pkg/parser/format",
|
||||
"//pkg/parser/mysql",
|
||||
|
||||
@ -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 <plugin>' but no 'BY <password>' 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
|
||||
|
||||
@ -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`)"},
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
|
||||
Reference in New Issue
Block a user