*: Avoid store plaintext password in mysql.user table.
Store hashed password.
This commit is contained in:
38
session.go
38
session.go
@ -19,7 +19,6 @@ package tidb
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/sha1"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"strings"
|
||||
@ -39,6 +38,7 @@ import (
|
||||
"github.com/pingcap/tidb/sessionctx/variable"
|
||||
"github.com/pingcap/tidb/stmt"
|
||||
"github.com/pingcap/tidb/stmt/stmts"
|
||||
"github.com/pingcap/tidb/util"
|
||||
"github.com/pingcap/tidb/util/errors2"
|
||||
)
|
||||
|
||||
@ -382,35 +382,6 @@ func (s *session) Close() error {
|
||||
return s.FinishTxn(true)
|
||||
}
|
||||
|
||||
func calcPassword(scramble, password []byte) []byte {
|
||||
if len(password) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// stage1Hash = SHA1(password)
|
||||
crypt := sha1.New()
|
||||
crypt.Write(password)
|
||||
stage1 := crypt.Sum(nil)
|
||||
|
||||
// scrambleHash = SHA1(scramble + SHA1(stage1Hash))
|
||||
// inner Hash
|
||||
crypt.Reset()
|
||||
crypt.Write(stage1)
|
||||
hash := crypt.Sum(nil)
|
||||
|
||||
// outer Hash
|
||||
crypt.Reset()
|
||||
crypt.Write(scramble)
|
||||
crypt.Write(hash)
|
||||
scramble = crypt.Sum(nil)
|
||||
|
||||
// token = scrambleHash XOR stage1Hash
|
||||
for i := range scramble {
|
||||
scramble[i] ^= stage1[i]
|
||||
}
|
||||
return scramble
|
||||
}
|
||||
|
||||
func (s *session) Auth(user string, auth []byte, salt []byte) bool {
|
||||
strs := strings.Split(user, "@")
|
||||
if len(strs) != 2 {
|
||||
@ -441,7 +412,12 @@ func (s *session) Auth(user string, auth []byte, salt []byte) bool {
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
checkAuth := calcPassword(salt, []byte(pwd))
|
||||
hpwd, err := util.DecodePassword(pwd)
|
||||
if err != nil {
|
||||
log.Errorf("Decode password string error %v", err)
|
||||
return false
|
||||
}
|
||||
checkAuth := util.CalcPassword(salt, hpwd)
|
||||
if !bytes.Equal(auth, checkAuth) {
|
||||
return false
|
||||
}
|
||||
|
||||
@ -31,6 +31,7 @@ import (
|
||||
"github.com/pingcap/tidb/rset/rsets"
|
||||
"github.com/pingcap/tidb/stmt"
|
||||
"github.com/pingcap/tidb/table"
|
||||
"github.com/pingcap/tidb/util"
|
||||
"github.com/pingcap/tidb/util/format"
|
||||
)
|
||||
|
||||
@ -139,10 +140,10 @@ func (s *CreateUserStmt) Exec(ctx context.Context) (rset.Recordset, error) {
|
||||
value = append(value, expression.Value{Val: host})
|
||||
value = append(value, expression.Value{Val: userName})
|
||||
if spec.AuthOpt.ByAuthString {
|
||||
value = append(value, expression.Value{Val: spec.AuthOpt.AuthString})
|
||||
value = append(value, expression.Value{Val: util.EncodePassword(spec.AuthOpt.AuthString)})
|
||||
} else {
|
||||
// TODO: Maybe we should hash the string here?
|
||||
value = append(value, expression.Value{Val: spec.AuthOpt.HashString})
|
||||
value = append(value, expression.Value{Val: util.EncodePassword(spec.AuthOpt.HashString)})
|
||||
}
|
||||
values = append(values, value)
|
||||
}
|
||||
@ -195,7 +196,7 @@ func (s *SetPwdStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) {
|
||||
// Update mysql.user
|
||||
asgn := expression.Assignment{
|
||||
ColName: "Password",
|
||||
Expr: expression.Value{Val: s.Password},
|
||||
Expr: expression.Value{Val: util.EncodePassword(s.Password)},
|
||||
}
|
||||
st := &UpdateStmt{
|
||||
TableRefs: composeUserTableRset(),
|
||||
|
||||
@ -15,6 +15,7 @@ package stmts_test
|
||||
|
||||
import (
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/util"
|
||||
)
|
||||
|
||||
func (s *testStmtSuite) TestCreateUserStmt(c *C) {
|
||||
@ -35,7 +36,7 @@ func (s *testStmtSuite) TestCreateUserStmt(c *C) {
|
||||
rows.Next()
|
||||
var pwd string
|
||||
rows.Scan(&pwd)
|
||||
c.Assert(pwd, Equals, "123")
|
||||
c.Assert(pwd, Equals, util.EncodePassword("123"))
|
||||
c.Assert(rows.Next(), IsFalse)
|
||||
rows.Close()
|
||||
mustCommit(c, tx)
|
||||
@ -72,7 +73,7 @@ func (s *testStmtSuite) TestSetPwdStmt(c *C) {
|
||||
c.Assert(err, IsNil)
|
||||
rows.Next()
|
||||
rows.Scan(&pwd)
|
||||
c.Assert(pwd, Equals, "password")
|
||||
c.Assert(pwd, Equals, util.EncodePassword("password"))
|
||||
c.Assert(rows.Next(), IsFalse)
|
||||
rows.Close()
|
||||
mustCommit(c, tx)
|
||||
|
||||
72
util/auth.go
Normal file
72
util/auth.go
Normal file
@ -0,0 +1,72 @@
|
||||
// Copyright 2013 The ql Authors. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSES/QL-LICENSE file.
|
||||
|
||||
// Copyright 2015 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,
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package util
|
||||
|
||||
import (
|
||||
"crypto/sha1"
|
||||
"encoding/hex"
|
||||
|
||||
"github.com/juju/errors"
|
||||
)
|
||||
|
||||
// CalcPassword is the algorithm convert hashed password to auth string.
|
||||
// See: https://dev.mysql.com/doc/internals/en/secure-password-authentication.html
|
||||
// SHA1( password ) XOR SHA1( "20-bytes random data from server" <concat> SHA1( SHA1( password ) ) )
|
||||
func CalcPassword(scramble, password []byte) []byte {
|
||||
if len(password) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// scrambleHash = SHA1(scramble + SHA1(stage1Hash))
|
||||
stage1 := password
|
||||
// inner Hash
|
||||
hash := Sha1Hash(stage1)
|
||||
// outer Hash
|
||||
crypt := sha1.New()
|
||||
crypt.Write(scramble)
|
||||
crypt.Write(hash)
|
||||
scramble = crypt.Sum(nil)
|
||||
|
||||
// token = scrambleHash XOR stage1Hash
|
||||
for i := range scramble {
|
||||
scramble[i] ^= stage1[i]
|
||||
}
|
||||
return scramble
|
||||
}
|
||||
|
||||
// Sha1Hash is an util function to calculate sha1 hash.
|
||||
func Sha1Hash(bs []byte) []byte {
|
||||
crypt := sha1.New()
|
||||
crypt.Write(bs)
|
||||
return crypt.Sum(nil)
|
||||
}
|
||||
|
||||
// EncodePassword converts plaintext password to hashed hex string.
|
||||
func EncodePassword(pwd string) string {
|
||||
hash := Sha1Hash([]byte(pwd))
|
||||
return hex.EncodeToString(hash)
|
||||
}
|
||||
|
||||
// DecodePassword converts hex string password to byte array.
|
||||
func DecodePassword(pwd string) ([]byte, error) {
|
||||
x, err := hex.DecodeString(pwd)
|
||||
if err != nil {
|
||||
return []byte{}, errors.Trace(err)
|
||||
}
|
||||
return x, nil
|
||||
}
|
||||
Reference in New Issue
Block a user