*: Avoid store plaintext password in mysql.user table.

Store hashed password.
This commit is contained in:
Shen Li
2015-10-04 22:31:37 +08:00
parent f90e031fb4
commit 948fa8bc2b
4 changed files with 86 additions and 36 deletions

View File

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

View File

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

View File

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