*: fix a bug for default_authentication_plugin (#57391)

ref pingcap/tidb#54138
This commit is contained in:
CbcWestwolf
2025-01-06 18:28:46 +08:00
committed by GitHub
parent 1756a8dae0
commit 7063a6fce0
15 changed files with 215 additions and 151 deletions

View File

@ -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

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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{

View File

@ -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)
}

View File

@ -52,7 +52,6 @@ go_test(
shard_count = 50,
deps = [
"//pkg/parser",
"//pkg/parser/auth",
"//pkg/parser/charset",
"//pkg/parser/format",
"//pkg/parser/mysql",

View File

@ -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

View File

@ -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`)"},

View File

@ -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

View File

@ -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

View File

@ -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 {

View File

@ -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)

View File

@ -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
}

View File

@ -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;

View File

@ -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;